mridulm commented on a change in pull request #34122:
URL: https://github.com/apache/spark/pull/34122#discussion_r794997274



##########
File path: core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
##########
@@ -855,7 +855,7 @@ class MapOutputTrackerSuite extends SparkFunSuite with 
LocalSparkContext {
     rpcEnv.shutdown()
   }
 
-  test("SPARK-37023: Avoid fetching merge status when shuffleMergeEnabled is 
false") {
+  test("SPARK-37023: Avoid fetching merge status when 
isShuffleMergeFinalizedMarked is false") {

Review comment:
       nit: isShuffleMergeFinalizedMarked -> useMergeResult ?
   That is what we are testing here actually (this test went through some 
changes as code evolved, but looks like test name was missed).

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -4051,8 +4050,8 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     runEvent(StageCancelled(0, Option("Explicit cancel check")))
     scheduler.handleShuffleMergeFinalized(shuffleStage1, 
shuffleStage1.shuffleDep.shuffleMergeId)
 
-    assert(shuffleStage1.shuffleDep.shuffleMergeEnabled)
-    assert(!shuffleStage1.shuffleDep.shuffleMergeFinalized)
+    assert(shuffleStage1.shuffleDep.mergerLocs.nonEmpty)
+    assert(!shuffleStage1.shuffleDep.isShuffleMergeFinalizedMarked)

Review comment:
       nit: revert ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -4147,7 +4146,206 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     assert(finalizeTask2.delay == 0 && finalizeTask2.registerMergeResults)
   }
 
-    /**
+  test("SPARK-34826: Adaptively fetch shuffle mergers") {
+    initPushBasedShuffleConfs(conf)
+    conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 2)
+    DAGSchedulerSuite.clearMergerLocs()
+    DAGSchedulerSuite.addMergerLocs(Seq("host1"))
+    val parts = 2
+
+    val shuffleMapRdd = new MyRDD(sc, parts, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, new 
HashPartitioner(parts))
+    val reduceRdd = new MyRDD(sc, parts, List(shuffleDep), tracker = 
mapOutputTracker)
+
+    // Submit a reduce job that depends which will create a map stage
+    submit(reduceRdd, (0 until parts).toArray)
+
+    runEvent(makeCompletionEvent(
+      taskSets(0).tasks(0), Success, makeMapStatus("hostA", parts),
+      Seq.empty, Array.empty, createFakeTaskInfoWithId(0)))
+
+    val shuffleStage1 = 
scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage]
+    assert(!shuffleStage1.shuffleDep.shuffleMergeEnabled)
+    assert(mapOutputTracker.getShufflePushMergerLocations(0).isEmpty)
+
+    DAGSchedulerSuite.addMergerLocs(Seq("host2", "host3"))
+
+    // host2 executor added event to trigger registering of shuffle merger 
locations
+    // as shuffle mergers are tracked separately for test
+    runEvent(ExecutorAdded("exec2", "host2"))
+
+    // Check if new shuffle merger locations are available for push or not
+    assert(mapOutputTracker.getShufflePushMergerLocations(0).size == 2)
+    assert(shuffleStage1.shuffleDep.getMergerLocs.size == 2)
+
+    // Complete remaining tasks in ShuffleMapStage 0
+    runEvent(makeCompletionEvent(taskSets(0).tasks(1), Success,
+      makeMapStatus("host1", parts), Seq.empty, Array.empty, 
createFakeTaskInfoWithId(1)))
+
+    completeNextResultStageWithSuccess(1, 0)
+    assert(results === Map(0 -> 42, 1 -> 42))
+
+    results.clear()
+    assertDataStructuresEmpty()
+  }
+
+  test("SPARK-34826: Adaptively fetch shuffle mergers with stage retry") {
+    initPushBasedShuffleConfs(conf)
+    conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 2)
+    DAGSchedulerSuite.clearMergerLocs()
+    DAGSchedulerSuite.addMergerLocs(Seq("host1"))
+    val parts = 2
+
+    val shuffleMapRdd1 = new MyRDD(sc, parts, Nil)
+    val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new 
HashPartitioner(parts))
+    val shuffleMapRdd2 = new MyRDD(sc, parts, Nil)
+    val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new 
HashPartitioner(parts))
+    val reduceRdd = new MyRDD(sc, parts, List(shuffleDep1, shuffleDep2),
+      tracker = mapOutputTracker)

Review comment:
       Based on discussion 
[above](https://github.com/apache/spark/pull/34122/files#r790376895), we might 
need to end up merging this test with the next ("SPARK-34826: Adaptively fetch 
shuffle mergers with stage retry for indeterminate stage").

##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -1281,6 +1341,33 @@ private[spark] class MapOutputTrackerWorker(conf: 
SparkConf) extends MapOutputTr
     }
   }
 
+  override def getShufflePushMergerLocations(shuffleId: Int): 
Seq[BlockManagerId] = {
+    shufflePushMergerLocations.getOrElse(shuffleId, 
getMergerLocations(shuffleId))
+  }
+
+  private def getMergerLocations(shuffleId: Int): Seq[BlockManagerId] = {
+    val mergers = shufflePushMergerLocations.get(shuffleId).orNull
+    if (mergers == null) {

Review comment:
       nit:
   This (get and null check) is not required - we are invoking this method only 
when `shufflePushMergerLocations` does not have an entry.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##########
@@ -728,7 +728,8 @@ private[spark] class DAGScheduler(
                 // Mark mapStage as available with shuffle outputs only after 
shuffle merge is
                 // finalized with push based shuffle. If not, subsequent 
ShuffleMapStage won't
                 // read from merged output as the MergeStatuses are not 
available.
-                if (!mapStage.isAvailable || 
!mapStage.shuffleDep.shuffleMergeFinalized) {
+                if (!mapStage.isAvailable ||
+                  !mapStage.shuffleDep.shuffleMergeFinalized) {

Review comment:
       nit: revert ?

##########
File path: 
core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala
##########
@@ -59,13 +59,25 @@ private[spark] class ShuffleWriteProcessor extends 
Serializable with Logging {
         rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: 
Product2[Any, Any]]])
       val mapStatus = writer.stop(success = true)
       if (mapStatus.isDefined) {
+        // Check if sufficient shuffle mergers are available now for the 
ShuffleMapTask to push
+        if (dep.shuffleMergeAllowed && dep.getMergerLocs.isEmpty) {
+          val mapOutputTracker = SparkEnv.get.mapOutputTracker
+          val mergerLocs =
+            mapOutputTracker.getShufflePushMergerLocations(dep.shuffleId)
+          if (mergerLocs.nonEmpty) {
+            dep.setMergerLocs(mergerLocs)
+          }
+        }
         // Initiate shuffle push process if push based shuffle is enabled
         // The map task only takes care of converting the shuffle data file 
into multiple
         // block push requests. It delegates pushing the blocks to a different 
thread-pool -
         // ShuffleBlockPusher.BLOCK_PUSHER_POOL.
-        if (dep.shuffleMergeEnabled && dep.getMergerLocs.nonEmpty && 
!dep.shuffleMergeFinalized) {
+        if (dep.getMergerLocs.nonEmpty && !dep.isShuffleMergeFinalizedMarked) {

Review comment:
       With the recent changes, wont `!dep.shuffleMergeFinalized` not be 
sufficient ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -4082,7 +4081,7 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     assert(shuffleIndeterminateStage.isIndeterminate)
     scheduler.handleShuffleMergeFinalized(shuffleIndeterminateStage, 2)
     assert(shuffleIndeterminateStage.shuffleDep.shuffleMergeEnabled)
-    assert(!shuffleIndeterminateStage.shuffleDep.shuffleMergeFinalized)
+    assert(!shuffleIndeterminateStage.shuffleDep.isShuffleMergeFinalizedMarked)
   }

Review comment:
       nit: revert ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -3926,7 +3925,7 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     val finalizeTask1 = shuffleStage1.shuffleDep.getFinalizeTask.get
       .asInstanceOf[DummyScheduledFuture]
     assert(finalizeTask1.delay == 10 && finalizeTask1.registerMergeResults)
-    assert(shuffleStage1.shuffleDep.shuffleMergeFinalized)
+    assert(shuffleStage1.shuffleDep.isShuffleMergeFinalizedMarked)

Review comment:
       nit: revert ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -3613,8 +3613,8 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     val shuffleStage2 = 
scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage]
     assert(shuffleStage2.shuffleDep.getMergerLocs.nonEmpty)
 
-    assert(shuffleStage2.shuffleDep.shuffleMergeFinalized)
-    assert(shuffleStage1.shuffleDep.shuffleMergeFinalized)
+    assert(shuffleStage2.shuffleDep.isShuffleMergeFinalizedMarked)
+    assert(shuffleStage1.shuffleDep.isShuffleMergeFinalizedMarked)

Review comment:
       nit: Revert these ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -3671,7 +3671,7 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
 
     completeShuffleMapStageSuccessfully(0, 0, parts)
     val shuffleStage = 
scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage]
-    assert(!shuffleStage.shuffleDep.shuffleMergeEnabled)
+    assert(shuffleStage.shuffleDep.mergerLocs.isEmpty)

Review comment:
       check for `!shuffleMergeAllowed` in addition to empty mergers here ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -4130,7 +4129,7 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     }
     val shuffleStage1 = 
scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage]
     assert(shuffleStage1.shuffleDep.shuffleMergeEnabled)
-    assert(!shuffleStage1.shuffleDep.shuffleMergeFinalized)
+    assert(!shuffleStage1.shuffleDep.isShuffleMergeFinalizedMarked)

Review comment:
       nit: revert ?

##########
File path: 
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -3686,14 +3686,13 @@ class DAGSchedulerSuite extends SparkFunSuite with 
TempLocalSparkContext with Ti
     completeNextStageWithFetchFailure(3, 0, shuffleDep)
     scheduler.resubmitFailedStages()
 
-    // Make sure shuffle merge is disabled for the retry
     val stage2 = scheduler.stageIdToStage(2).asInstanceOf[ShuffleMapStage]
-    assert(!stage2.shuffleDep.shuffleMergeEnabled)
+    assert(stage2.shuffleDep.shuffleMergeEnabled)

Review comment:
       I was thinking more about this ... should we always mark DETERMINATE 
stages with `setShuffleMergeAllowed(false)` when they complete ?
   It makes things more cleaner, and in line with expectation whether mergers 
were used for first execution or not.
   
   Thoughts ?

##########
File path: 
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
##########
@@ -131,7 +131,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) 
extends ShuffleManager
       metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = {
     val baseShuffleHandle = handle.asInstanceOf[BaseShuffleHandle[K, _, C]]
     val (blocksByAddress, canEnableBatchFetch) =
-      if (baseShuffleHandle.dependency.shuffleMergeEnabled) {
+      if (baseShuffleHandle.dependency.isShuffleMergeFinalizedMarked) {

Review comment:
       Given the changes, should this be `shuffleMergeEnabled && 
isShuffleMergeFinalizedMarked ` ?




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to