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 2021/12/01 07:14:16 UTC

[GitHub] [spark] mridulm commented on a change in pull request #33896: [SPARK-33701][SHUFFLE] Adaptive shuffle merge finalization for push-based shuffle

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



##########
File path: core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -3847,6 +3887,76 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
 
     // Job successful ended.
     assert(results === Map(0 -> 11, 1 -> 12))
+  }
+
+  test("SPARK-33701: shuffle adaptive merge finalization") {
+    initPushBasedShuffleConfs(conf)
+    conf.set(config.PUSH_BASED_SHUFFLE_SIZE_MIN_SHUFFLE_SIZE_TO_WAIT, 10L)
+    conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+    DAGSchedulerSuite.clearMergerLocs
+    DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3", "host4", "host5"))
+    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)
+
+    // Submit a reduce job that depends which will create a map stage
+    submit(reduceRdd, (0 until parts).toArray)
+
+    val taskResults = taskSets(0).tasks.zipWithIndex.map {
+      case (_, idx) =>
+        (Success, makeMapStatus("host" + ('A' + idx).toChar, parts))
+    }.toSeq
+    // Remove MapStatus on one of the host before the stage ends to trigger
+    // a scenario where stage 0 needs to be resubmitted upon finishing all tasks.
+    // Merge finalization should not be scheduled in this case.
+    for ((result, i) <- taskResults.zipWithIndex) {
+      if (i == taskSets(0).tasks.size - 1) {
+        mapOutputTracker.removeOutputsOnHost("hostA")
+      }
+      if (i < taskSets(0).tasks.size) {
+        runEvent(makeCompletionEvent(taskSets(0).tasks(i), result._1, result._2))
+      }
+    }

Review comment:
       This snippet is a bit confusing.
   
   a) We can move the `taskSets(0).tasks.size - 1` out of the loop ?
   b) `i` will always be less than `tasks.size` - why the if condition ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -3847,6 +3887,76 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
 
     // Job successful ended.
     assert(results === Map(0 -> 11, 1 -> 12))
+  }
+
+  test("SPARK-33701: shuffle adaptive merge finalization") {
+    initPushBasedShuffleConfs(conf)
+    conf.set(config.PUSH_BASED_SHUFFLE_SIZE_MIN_SHUFFLE_SIZE_TO_WAIT, 10L)
+    conf.set(config.SHUFFLE_MERGER_LOCATIONS_MIN_STATIC_THRESHOLD, 3)
+    DAGSchedulerSuite.clearMergerLocs
+    DAGSchedulerSuite.addMergerLocs(Seq("host1", "host2", "host3", "host4", "host5"))
+    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)
+
+    // Submit a reduce job that depends which will create a map stage
+    submit(reduceRdd, (0 until parts).toArray)
+
+    val taskResults = taskSets(0).tasks.zipWithIndex.map {
+      case (_, idx) =>
+        (Success, makeMapStatus("host" + ('A' + idx).toChar, parts))
+    }.toSeq
+    // Remove MapStatus on one of the host before the stage ends to trigger
+    // a scenario where stage 0 needs to be resubmitted upon finishing all tasks.
+    // Merge finalization should not be scheduled in this case.
+    for ((result, i) <- taskResults.zipWithIndex) {
+      if (i == taskSets(0).tasks.size - 1) {
+        mapOutputTracker.removeOutputsOnHost("hostA")
+      }
+      if (i < taskSets(0).tasks.size) {
+        runEvent(makeCompletionEvent(taskSets(0).tasks(i), result._1, result._2))
+      }
+    }
+    val shuffleStage1 = scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage]
+    // Successfully completing the retry of stage 0. Merge finalization should be
+    // disabled
+     complete(taskSets(2), taskSets(2).tasks.zipWithIndex.map {
+      case (_, idx) =>
+        (Success, makeMapStatus("host" + ('A' + idx).toChar, parts))
+    }.toSeq)
+    assert(!shuffleStage1.shuffleDep.shuffleMergeEnabled)
+    // Verify finalize task is set with 0 delay and merge results not marked
+    // for registration due to shuffle size smaller than threshold
+    assert(shuffleStage1.shuffleDep.getFinalizeTask.nonEmpty)

Review comment:
       When shuffleMerge is disabled, why is finalize task getting set ?
   We should be checking for expected behavior (not specifics of how the code is right now - given the specific impl details can evolve).

##########
File path: core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala
##########
@@ -327,10 +338,32 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging {
         s"stop.")
       return false
     } else {
+      if (reqsInFlight <= 0 && pushRequests.isEmpty && deferredPushRequests.isEmpty) {

Review comment:
       Instead of `FakeShuffleClient`, we can mock `shuffleClient.pushBlocks` such that for specific host:port, we delay responding.
   For example:
   a) For requests to host1, we wait on a latch.
   b) For other requests we return immediately (as in `interceptPushedBlocksForSuccess`).
   
   This can be used to check all blocks are in flight, some have completed and `reqsInFlight > 0` ?
   




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