spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From shivaram <...@git.apache.org>
Subject [GitHub] spark pull request: [SPARK-3019] Pluggable block transfer interfac...
Date Wed, 03 Sep 2014 03:32:41 GMT
Github user shivaram commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2240#discussion_r17029964
  
    --- Diff: core/src/main/scala/org/apache/spark/network/cm/CMBlockTransferService.scala
---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.network.cm
    +
    +import java.nio.ByteBuffer
    +
    +import scala.concurrent.Await
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf}
    +import org.apache.spark.network._
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.util.Utils
    +
    +
    +/**
    + * A [[BlockTransferService]] implementation based on our [[ConnectionManager]].
    + */
    +final class CMBlockTransferService(conf: SparkConf, securityManager: SecurityManager)
    +  extends BlockTransferService with Logging {
    +
    +  private var cm: ConnectionManager = _
    +
    +  private var blockDataManager: BlockDataManager = _
    +
    +  /**
    +   * Port number the service is listening on, available only after [[init]] is invoked.
    +   */
    +  override def port: Int = {
    +    checkInit()
    +    cm.id.port
    +  }
    +
    +  /**
    +   * Host name the service is listening on, available only after [[init]] is invoked.
    +   */
    +  override def hostName: String = {
    +    checkInit()
    +    cm.id.host
    +  }
    +
    +  /**
    +   * Initialize the transfer service by giving it the BlockDataManager that can be used
to fetch
    +   * local blocks or put local blocks.
    +   */
    +  override def init(blockDataManager: BlockDataManager): Unit = {
    +    this.blockDataManager = blockDataManager
    +    cm = new ConnectionManager(
    +      conf.getInt("spark.blockManager.port", 0),
    +      conf,
    +      securityManager,
    +      "Connection manager for block manager")
    +    cm.onReceiveMessage(onBlockMessageReceive)
    +  }
    +
    +  /**
    +   * Tear down the transfer service.
    +   */
    +  override def stop(): Unit = {
    +    if (cm != null) {
    +      cm.stop()
    +    }
    +  }
    +
    +  override def fetchBlocks(
    +      hostName: String,
    +      port: Int,
    +      blockIds: Seq[String],
    +      listener: BlockFetchingListener): Unit = {
    +    checkInit()
    +
    +    val cmId = new ConnectionManagerId(hostName, port)
    +    val blockMessageArray = new BlockMessageArray(blockIds.map { blockId =>
    +      BlockMessage.fromGetBlock(GetBlock(BlockId(blockId)))
    +    })
    +
    +    val future = cm.sendMessageReliably(cmId, blockMessageArray.toBufferMessage)
    +
    +    // Register the listener on success/failure future callback.
    +    future.onSuccess { case message =>
    +      val bufferMessage = message.asInstanceOf[BufferMessage]
    +      val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage)
    +
    +      for (blockMessage <- blockMessageArray) {
    +        if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) {
    +          listener.onBlockFetchFailure(
    +            new SparkException(s"Unexpected message ${blockMessage.getType} received
from $cmId"))
    +        } else {
    +          val blockId = blockMessage.getId
    +          val networkSize = blockMessage.getData.limit()
    +          listener.onBlockFetchSuccess(
    +            blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData))
    +        }
    +      }
    +    }(cm.futureExecContext)
    +
    +    future.onFailure { case exception =>
    +      listener.onBlockFetchFailure(exception)
    +    }(cm.futureExecContext)
    +  }
    +
    +  /**
    +   * Upload a single block to a remote node, available only after [[init]] is invoked.
    +   *
    +   * This call blocks until the upload completes, or throws an exception upon failures.
    +   */
    +  override def uploadBlock(
    +      hostname: String,
    +      port: Int,
    +      blockId: String,
    +      blockData: ManagedBuffer,
    +      level: StorageLevel) {
    +    checkInit()
    +    val msg = PutBlock(BlockId(blockId), blockData.byteBuffer(), level)
    +    val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg))
    +    val remoteCmId = new ConnectionManagerId(hostName, port)
    +
    +    // TODO: Not wait infinitely.
    --- End diff --
    
    this would go away if we had a listener interface here ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Mime
View raw message