Github user squito commented on a diff in the pull request:
https://github.com/apache/spark/pull/16620#discussion_r97417513
--- Diff:
core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
---
@@ -648,4 +648,69 @@ class BasicSchedulerIntegrationSuite extends
SchedulerIntegrationSuite[SingleCor
}
assertDataStructuresEmpty(noFailure = false)
}
+
+ testScheduler("[SPARK-19263] DAGScheduler shouldn't resubmit active
taskSet.") {
+ val a = new MockRDD(sc, 2, Nil)
+ val b = shuffle(2, a)
+ val shuffleId = b.shuffleDeps.head.shuffleId
+
+ def runBackend(): Unit = {
+ val (taskDescription, task) = backend.beginTask()
+ task.stageId match {
+ // ShuffleMapTask
+ case 0 =>
+ val stageAttempt = task.stageAttemptId
+ val partitionId = task.partitionId
+ (stageAttempt, partitionId) match {
+ case (0, 0) =>
+ val fetchFailed = FetchFailed(
+ DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId,
0, 0, "ignored")
+ backend.taskFailed(taskDescription, fetchFailed)
+ case (0, 1) =>
+ // Wait until stage resubmission caused by FetchFailed is
finished.
+
waitUntilConditionBecomeTrue(taskScheduler.runningTaskSets.size==2, 5000,
+ "Wait until stage is resubmitted caused by fetch failed")
+
+ // Task(stageAttempt=0, partition=1) will be bogus, because
both two
+ // tasks(stageAttempt=0, partition=0, 1) run on hostA.
+ // Pending partitions are (0, 1) after stage resubmission,
+ // then change to be 0 after this bogus task.
+ backend.taskSuccess(taskDescription,
DAGSchedulerSuite.makeMapStatus("hostA", 2))
+ case (1, 1) =>
+ // Wait long enough until Success of task(stageAttempt=1 and
partition=0)
+ // is handled by DAGScheduler.
+ Thread.sleep(5000)
+ // Task(stageAttempt=1 and partition=0) will cause stage
resubmission,
+ // because shuffleStage.pendingPartitions.isEmpty,
+ // but shuffleStage.isAvailable is false.
+ backend.taskSuccess(taskDescription,
DAGSchedulerSuite.makeMapStatus("hostB", 2))
+ case _ =>
+ backend.taskSuccess(taskDescription,
DAGSchedulerSuite.makeMapStatus("hostB", 2))
+ }
+ // ResultTask
+ case 1 => backend.taskSuccess(taskDescription, 10)
+ }
+ }
+
+ withBackend(runBackend _) {
+ val jobFuture = submit(b, (0 until 2).toArray)
+ val duration = Duration(15, SECONDS)
+ awaitJobTermination(jobFuture, duration)
+ }
+ assert(results === (0 until 2).map { _ -> 10}.toMap)
+ }
+
+ def waitUntilConditionBecomeTrue(condition: => Boolean, timeout: Long,
msg: String): Unit = {
--- End diff --
nit: rename to `waitForCondition` (maybe irrevlant given other comments)
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]