You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/08/27 15:43:28 UTC

[GitHub] [spark] tgravescs commented on a change in pull request #25299: [SPARK-27651][Core] Avoid the network when shuffle blocks are fetched from the same host

tgravescs commented on a change in pull request #25299: [SPARK-27651][Core] Avoid the network when shuffle blocks are fetched from the same host
URL: https://github.com/apache/spark/pull/25299#discussion_r318147781
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
 ##########
 @@ -358,12 +382,65 @@ final class ShuffleBlockFetcherIterator(
     }
   }
 
+
+  /**
+   * Fetch the host-local blocks while we are fetching remote blocks. This is ok because
+   * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we
+   * track in-memory are the ManagedBuffer references themselves.
+   */
+  private[this] def fetchHostLocalBlocks() {
+    logDebug(s"Start fetching host-local blocks: ${hostLocalBlocks.mkString(", ")}")
+    val hostLocalExecutorIds = hostLocalBlocksByExecutor.keySet.map(_.executorId)
+    val readsWithoutLocalDir = LinkedHashMap[BlockManagerId, Seq[(BlockId, Long)]]()
+    val localDirsByExec = blockManager.getHostLocalDirs(hostLocalExecutorIds.toArray)
+    hostLocalBlocksByExecutor.foreach { case (bmId, blockInfos) =>
+      val localDirs = localDirsByExec.get(bmId.executorId)
+      if (localDirs.isDefined) {
+        blockInfos.foreach {  case (blockId, _) =>
+          try {
+            val buf = blockManager
+              .getHostLocalShuffleData(blockId.asInstanceOf[ShuffleBlockId], localDirs.get)
 
 Review comment:
   I assume our failure case here is the same as the network timeouts then?  I'm wondering about the thundering herd of all the tasks hitting the driver all at once and it being slow to respond and thus things timeout. I'm assuming that is resulting in the fetch failure below. , theoretically if this failed we could fall back to fetch remote.  that may or may not just fail again depending on if it was the thundering herd problem or a real issue.
   not sure we want to do it here, I guess we can wait and see if a problem, hopefully with the cache its pretty quick.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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