Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2366#discussion_r17906928
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala 
---
    @@ -787,31 +789,88 @@ private[spark] class BlockManager(
       }
     
       /**
    +   * Get peer block managers in the system.
    +   */
    +  private def getPeers(forceFetch: Boolean): HashSet[BlockManagerId] = {
    +    val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 
1000) // milliseconds
    +    val timeout = System.currentTimeMillis - lastPeerFetchTime > 
cachedPeersTtl
    +
    +    cachedPeers.synchronized {
    +      if (cachedPeers.isEmpty || forceFetch || timeout) {
    +        cachedPeers.clear()
    +        cachedPeers ++= master.getPeers(blockManagerId).sortBy(_.hashCode)
    +        lastPeerFetchTime = System.currentTimeMillis
    +        logDebug("Fetched peers from master: " + cachedPeers.mkString("[", 
",", "]"))
    +      }
    +    }
    +    cachedPeers
    +  }
    +
    +  /**
        * Replicate block to another node.
        */
    -  @volatile var cachedPeers: Seq[BlockManagerId] = null
       private def replicate(blockId: BlockId, data: ByteBuffer, level: 
StorageLevel): Unit = {
    +    val maxReplicationFailures = 
conf.getInt("spark.storage.maxReplicationFailures", 1)
    +    val numPeersToReplicateTo = level.replication - 1
    +    val peersReplicatedTo = new HashSet[BlockManagerId]
    +    val peersFailedToReplicateTo = new HashSet[BlockManagerId]
         val tLevel = StorageLevel(
           level.useDisk, level.useMemory, level.useOffHeap, 
level.deserialized, 1)
    -    if (cachedPeers == null) {
    -      cachedPeers = master.getPeers(blockManagerId, level.replication - 1)
    +    val startTime = System.nanoTime
    +    val random = new Random(blockId.hashCode)
    +
    +    var forceFetchPeers = false
    +    var failures = 0
    +    var done = false
    +
    +    // Get a random peer
    +    def getRandomPeer(): Option[BlockManagerId] = {
    +      val peers = getPeers(forceFetchPeers) -- peersReplicatedTo -- 
peersFailedToReplicateTo
    +      if (!peers.isEmpty) Some(peers.toSeq(random.nextInt(peers.size))) 
else None
         }
    -    for (peer: BlockManagerId <- cachedPeers) {
    -      val start = System.nanoTime
    -      data.rewind()
    -      logDebug(s"Try to replicate $blockId once; The size of the data is 
${data.limit()} Bytes. " +
    -        s"To node: $peer")
     
    -      try {
    -        blockTransferService.uploadBlockSync(
    -          peer.host, peer.port, blockId.toString, new 
NioByteBufferManagedBuffer(data), tLevel)
    -      } catch {
    -        case e: Exception =>
    -          logError(s"Failed to replicate block to $peer", e)
    +    // One by one choose a random peer and try uploading the block to it
    +    // If replication fails (e.g., target peer is down), force the list of 
cached peers
    +    // to be re-fetched from driver and then pick another random peer for 
replication. Also
    +    // temporarily black list the peer for which replication failed.
    +    while (!done) {
    --- End diff --
    
    I added comments before the while, as well as at all the 3 places where 
`done` is marked as `true`.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to