venkata91 commented on a change in pull request #34122:
URL: https://github.com/apache/spark/pull/34122#discussion_r792325877
##########
File path:
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
##########
@@ -4147,7 +4146,210 @@ 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.getMergerLocs.isEmpty)
+ 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("host2", "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)
+
+ // 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" + idx, parts))
+ }.toSeq
+
+ val shuffleStage1 =
scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage]
+ 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("host2", "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)
+ val mergerLocsBeforeRetry = shuffleStage1.shuffleDep.getMergerLocs
+
+ // Clear merger locations to check if new mergers are not getting set for
the
+ // retry of determinate stage
+ DAGSchedulerSuite.clearMergerLocs()
+
+ // 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 be scheduled in this case.
+ for ((result, i) <- taskResults.zipWithIndex) {
+ if (i == taskSets(0).tasks.size - 1) {
+ mapOutputTracker.removeOutputsOnHost("host0")
+ }
+ if (i < taskSets(0).tasks.size) {
+ runEvent(makeCompletionEvent(taskSets(0).tasks(i), result._1,
result._2))
+ }
+ }
+ assert(shuffleStage1.shuffleDep.shuffleMergeFinalized)
+
+ DAGSchedulerSuite.addMergerLocs(Seq("host4", "host5"))
+ // host4 executor added event shouldn't reset merger locations given
merger locations
+ // are already set
+ runEvent(ExecutorAdded("host4", "host4"))
+
+ // Successfully completing the retry of stage 0.
+ complete(taskSets(2), taskSets(2).tasks.zipWithIndex.map {
+ case (_, idx) =>
+ (Success, makeMapStatus("host" + idx, parts))
+ }.toSeq)
+
+ assert(shuffleStage1.shuffleDep.shuffleMergeId == 0)
+ assert(shuffleStage1.shuffleDep.getMergerLocs.size == 2)
+ assert(shuffleStage1.shuffleDep.shuffleMergeFinalized)
+ val newMergerLocs =
+
scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage].shuffleDep.getMergerLocs
+ assert(mergerLocsBeforeRetry.sortBy(_.host) ===
newMergerLocs.sortBy(_.host))
+ val shuffleStage2 =
scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage]
+ complete(taskSets(1), taskSets(1).tasks.zipWithIndex.map {
+ case (_, idx) =>
+ (Success, makeMapStatus("host" + idx, parts, 10))
+ }.toSeq)
+ assert(shuffleStage2.shuffleDep.getMergerLocs.size == 2)
+ completeNextResultStageWithSuccess(2, 0)
+ assert(results === Map(0 -> 42, 1 -> 42))
+
+ results.clear()
+ assertDataStructuresEmpty()
+ }
+
+ test("SPARK-34826: Adaptively fetch shuffle mergers with stage retry for
indeterminate stage") {
+ 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, indeterminate = true)
+ val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new
HashPartitioner(parts))
+ val shuffleMapRdd2 = new MyRDD(sc, parts, Nil, indeterminate = true)
+ 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" + idx, parts))
+ }.toSeq
+
+ val shuffleStage1 =
scheduler.stageIdToStage(0).asInstanceOf[ShuffleMapStage]
+ 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("host2", "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)
+ val mergerLocsBeforeRetry = shuffleStage1.shuffleDep.getMergerLocs
+
+ // Clear merger locations to check if new mergers are getting set for the
+ // retry of indeterminate stage
+ DAGSchedulerSuite.clearMergerLocs()
+
+ // 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 be scheduled in this case.
+ for ((result, i) <- taskResults.zipWithIndex) {
+ if (i == taskSets(0).tasks.size - 1) {
+ mapOutputTracker.removeOutputsOnHost("host0")
+ }
+ if (i < taskSets(0).tasks.size) {
Review comment:
No, internally we had this and just carried over as part of forward
port. This is not required.
##########
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##########
@@ -1369,24 +1370,34 @@ private[spark] class DAGScheduler(
* locations for block push/merge by getting the historical locations of
past executors.
*/
private def prepareShuffleServicesForShuffleMapStage(stage:
ShuffleMapStage): Unit = {
- assert(stage.shuffleDep.shuffleMergeEnabled &&
!stage.shuffleDep.shuffleMergeFinalized)
+ assert(stage.shuffleDep.shuffleMergeAllowed &&
!stage.shuffleDep.shuffleMergeFinalized)
if (stage.shuffleDep.getMergerLocs.isEmpty) {
- val mergerLocs = sc.schedulerBackend.getShufflePushMergerLocations(
- stage.shuffleDep.partitioner.numPartitions, stage.resourceProfileId)
- if (mergerLocs.nonEmpty) {
- stage.shuffleDep.setMergerLocs(mergerLocs)
- logInfo(s"Push-based shuffle enabled for $stage (${stage.name}) with" +
- s" ${stage.shuffleDep.getMergerLocs.size} merger locations")
-
- logDebug("List of shuffle push merger locations " +
- s"${stage.shuffleDep.getMergerLocs.map(_.host).mkString(", ")}")
- } else {
- stage.shuffleDep.setShuffleMergeEnabled(false)
- logInfo(s"Push-based shuffle disabled for $stage (${stage.name})")
- }
+ getAndSetShufflePushMergerLocations(stage)
+ }
+
+ if (stage.shuffleDep.shuffleMergeEnabled) {
+ logInfo(("Shuffle merge enabled before starting the stage for %s (%s)
with %d" +
+ " merger locations").format(stage, stage.name,
stage.shuffleDep.getMergerLocs.size))
Review comment:
Added.
--
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]