You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "mridulm (via GitHub)" <gi...@apache.org> on 2023/08/12 01:14:30 UTC

[GitHub] [spark] mridulm commented on a diff in pull request #42296: [SPARK-44635][CORE] Handle shuffle fetch failures in decommissions

mridulm commented on code in PR #42296:
URL: https://github.com/apache/spark/pull/42296#discussion_r1291923436


##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -1288,6 +1288,30 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr
     mapSizesByExecutorId.iter
   }
 
+  def getMapOutputLocationWithRefresh(
+      shuffleId: Int,
+      mapId: Long,
+      prevLocation: BlockManagerId): BlockManagerId = {
+    // Try to get the cached location first in case other concurrent tasks
+    // fetched the fresh location already
+    var currentLocationOpt = getMapOutputLocation(shuffleId, mapId)
+    if (currentLocationOpt.isDefined && currentLocationOpt.get == prevLocation) {

Review Comment:
   nit:
   
   ```suggestion
       if (currentLocationOpt.exists(_ == prevLocation)) {
   ```



##########
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala:
##########
@@ -264,18 +272,22 @@ final class ShuffleBlockFetcherIterator(
       case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex))
     }.toMap
     val remainingBlocks = new HashSet[String]() ++= infoMap.keys
-    val deferredBlocks = new ArrayBuffer[String]()
+    val deferredBlocks = new HashMap[BlockManagerId, Queue[String]]()
     val blockIds = req.blocks.map(_.blockId.toString)
     val address = req.address
     val requestStartTime = clock.nanoTime()
 
     @inline def enqueueDeferredFetchRequestIfNecessary(): Unit = {
       if (remainingBlocks.isEmpty && deferredBlocks.nonEmpty) {
-        val blocks = deferredBlocks.map { blockId =>
-          val (size, mapIndex) = infoMap(blockId)
-          FetchBlockInfo(BlockId(blockId), size, mapIndex)
+        val newAddressToBlocks = new HashMap[BlockManagerId, Queue[FetchBlockInfo]]()
+        deferredBlocks.foreach { case (blockManagerId, blockIds) =>
+          val blocks = blockIds.map { blockId =>
+            val (size, mapIndex) = infoMap(blockId)
+            FetchBlockInfo(BlockId(blockId), size, mapIndex)
+          }
+          newAddressToBlocks.put(blockManagerId, blocks)
         }

Review Comment:
   Why not something like:
   ```
   deferredBlocks.foreach { case (blockManagerId, blockIds) =>
             val blocks = blockIds.map { blockId =>
               val (size, mapIndex) = infoMap(blockId)
               FetchBlockInfo(BlockId(blockId), size, mapIndex)
             }
             results.put(DeferFetchRequestResult(FetchRequest(blockManagerId, blocks)))
           }
   ```
   



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -528,6 +528,15 @@ package object config {
       .bytesConf(ByteUnit.BYTE)
       .createOptional
 
+  private[spark] val STORAGE_DECOMMISSION_SHUFFLE_REFRESH =
+    ConfigBuilder("spark.storage.decommission.shuffleBlocks.refreshLocationsEnabled")
+      .doc("If true, executors will try to refresh the cached locations for the shuffle blocks" +
+        "when fetch failures happens (and decommission shuffle block migration is enabled), " +
+        "and retry fetching when the location changes.")
+      .version("3.5.0")

Review Comment:
   Change to `4.0.0`



##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -1288,6 +1288,30 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr
     mapSizesByExecutorId.iter
   }
 
+  def getMapOutputLocationWithRefresh(
+      shuffleId: Int,
+      mapId: Long,
+      prevLocation: BlockManagerId): BlockManagerId = {
+    // Try to get the cached location first in case other concurrent tasks
+    // fetched the fresh location already
+    var currentLocationOpt = getMapOutputLocation(shuffleId, mapId)
+    if (currentLocationOpt.isDefined && currentLocationOpt.get == prevLocation) {
+      // Address in the cache unchanged. Try to clean cache and get a fresh location
+      unregisterShuffle(shuffleId)
+      currentLocationOpt = getMapOutputLocation(shuffleId, mapId)

Review Comment:
   Note: we end up removing both map and merge status here - for this call second call, pass `canFetchMergeResult = true` in ` getMapOutputLocation`



##########
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala:
##########
@@ -264,18 +272,22 @@ final class ShuffleBlockFetcherIterator(
       case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex))
     }.toMap
     val remainingBlocks = new HashSet[String]() ++= infoMap.keys
-    val deferredBlocks = new ArrayBuffer[String]()
+    val deferredBlocks = new HashMap[BlockManagerId, Queue[String]]()

Review Comment:
   nit:
   
   ```suggestion
       val deferredBlocks = new HashMap[BlockManagerId, ArrayBuffer[String]]()
   ```



##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -1288,6 +1288,30 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr
     mapSizesByExecutorId.iter
   }
 
+  def getMapOutputLocationWithRefresh(
+      shuffleId: Int,
+      mapId: Long,
+      prevLocation: BlockManagerId): BlockManagerId = {
+    // Try to get the cached location first in case other concurrent tasks
+    // fetched the fresh location already
+    var currentLocationOpt = getMapOutputLocation(shuffleId, mapId)
+    if (currentLocationOpt.isDefined && currentLocationOpt.get == prevLocation) {
+      // Address in the cache unchanged. Try to clean cache and get a fresh location
+      unregisterShuffle(shuffleId)
+      currentLocationOpt = getMapOutputLocation(shuffleId, mapId)
+    }
+    if (currentLocationOpt.isEmpty) {
+      throw new MetadataFetchFailedException(shuffleId, -1,
+        message = s"Failed to get map output location for shuffleId $shuffleId, mapId $mapId")
+    }
+    currentLocationOpt.get

Review Comment:
   nit: `currentLocationOpt.getOrElse( throw ... )`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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