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

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

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


##########
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:
   Good catch. Will do.



##########
core/src/main/scala/org/apache/spark/TestUtils.scala:
##########
@@ -491,6 +491,27 @@ private[spark] object TestUtils {
       EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE))
     file.getPath
   }
+
+  def withConf[T](confPairs: (String, String)*)(f: => T): T = {

Review Comment:
   Sure. Will do.



##########
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:
   Will change to `currentLocationOpt.contains(prevLocation)`.



##########
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(

Review Comment:
   We still want to throw a `MetadataFetchFailedException` when failing to get a refreshed location here. So I would prefer returning a `BlockManagerId` and make it specific.



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