mridulm commented on a change in pull request #34122:
URL: https://github.com/apache/spark/pull/34122#discussion_r796929787
##########
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##########
@@ -1821,7 +1834,7 @@ private[spark] class DAGScheduler(
}
if (runningStages.contains(shuffleStage) &&
shuffleStage.pendingPartitions.isEmpty) {
- if (!shuffleStage.shuffleDep.shuffleMergeFinalized &&
+ if (!shuffleStage.shuffleDep.isShuffleMergeFinalizedMarked &&
shuffleStage.shuffleDep.getMergerLocs.nonEmpty) {
checkAndScheduleShuffleMergeFinalize(shuffleStage)
} else {
Review comment:
In `processShuffleMapStageCompletion`, add something like
```
if (!shuffleStage.isIndeterminate &&
shuffleStage.shuffleDep.shuffleMergeEnabled) {
shuffleStage.shuffleDep.setShuffleMergeAllowed(false)
}
```
to ensure we dont retry merge for determinate stages ?
This is strictly not related to this PR, so I am fine with doing it in a
follow up PR as well to keep the scope contained (we can a jira in that case).
##########
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:
Looks like this was missed ?
##########
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:
Let us keep them separate, and do it with the other jira if we decide to
take that path.
##########
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##########
@@ -2313,7 +2326,7 @@ private[spark] class DAGScheduler(
// Register merge statuses if the stage is still running and shuffle merge
is not finalized yet.
// TODO: SPARK-35549: Currently merge statuses results which come after
shuffle merge
// TODO: is finalized is not registered.
- if (runningStages.contains(stage) &&
!stage.shuffleDep.shuffleMergeFinalized) {
+ if (runningStages.contains(stage) &&
!stage.shuffleDep.isShuffleMergeFinalizedMarked) {
Review comment:
QQ:
We are not checking if merge id is the same here.
Consider case where `finalizeShuffleMerge` takes 10s or so - and we have
stage cancellation/reexecution in the mean time.
We should do this for both `RegisterMergeStatuses` and
`ShuffleMergeFinalized`
Thoughts ? (We can file a follow up jira for this if valid)
--
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]