Github user squito commented on a diff in the pull request: https://github.com/apache/spark/pull/5636#discussion_r35801109 --- Diff: core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala --- @@ -473,6 +473,326 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } + // Helper function to validate state when creating tests for task failures + def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) { + assert(stageAttempt.stageId === stageId) + assert(stageAttempt.stageAttemptId == attempt) + } + + def makeCompletions(stageAttempt: TaskSet, reduceParts: Int): Seq[(Success.type, MapStatus)] = { + stageAttempt.tasks.zipWithIndex.map { case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, reduceParts)) + }.toSeq + } + + def setupStageAbortTest(sc: SparkContext) { + sc.listenerBus.addListener(new EndListener()) + ended = false + jobResult = null + } + + // Create a new Listener to confirm that the listenerBus sees the JobEnd message + // when we abort the stage. This message will also be consumed by the EventLoggingListener + // so this will propagate up to the user. + var ended = false + var jobResult : JobResult = null + + class EndListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + jobResult = jobEnd.jobResult + ended = true + } + } + + /** + * In this test we simulate a job failure where the first stage completes successfully and + * the second stage fails due to a fetch failure. Multiple successive fetch failures of a stage + * trigger an overall stage abort to avoid endless retries. + */ + test("Multiple consecutive stage failures should lead to task being aborted.") { + setupStageAbortTest(sc) + + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + + for (attempt <- 0 until Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last + checkStageId(0, attempt, stage0Attempt) + + // Run stage 0 + complete(stage0Attempt, makeCompletions(stage0Attempt, 2)) + + // Now we should have a new taskSet, for a new attempt of stage 1. + // We will have one fetch failure for this task set + val stage1Attempt = taskSets.last + checkStageId(1, attempt, stage1Attempt) + + val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, 42)} + + // Run Stage 1, this time with a task failure + complete(stage1Attempt, + Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) + ++ stage1Successes + ) + + // this will (potentially) trigger a resubmission of stage 0, since we've lost some of its + // map output, for the next iteration through the loop + scheduler.resubmitFailedStages() + + if (attempt < Stage.MAX_STAGE_FAILURES-1) { + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + } else { + // Stage has been aborted and removed from running stages + assertDataStructuresEmpty() + sc.listenerBus.waitUntilEmpty(1000) + assert(ended) + assert(jobResult.isInstanceOf[JobFailed]) + } + } + } + + /** + * In this test we simulate a job failure where there are two failures in two different stages. + * Specifically, stage0 fails twice, and then stage1 twice. In total, the job has had four + * failures overall but not four failures for a particular stage, and as such should not be + * aborted. + */ + test("Failures in different stages should not trigger an overall abort") { + setupStageAbortTest(sc) + + val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache() + val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)).cache() + val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) + submit(finalRdd, Array(0)) + + // In the first two iterations, Stage 0 succeeds and stage 1 fails. In the next two iterations, + // stage 2 fails. + for (attempt <- 0 until Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last + checkStageId(0, attempt, stage0Attempt) + // Run stage 0 + complete(stage0Attempt, makeCompletions(stage0Attempt, 2)) + + if (attempt < Stage.MAX_STAGE_FAILURES/2) { + // Now we should have a new taskSet, for a new attempt of stage 1. + // We will have one fetch failure for this task set + val stage1Attempt = taskSets.last + checkStageId(1, attempt, stage1Attempt) + + val stage1Successes = + stage1Attempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} + + // Run Stage 1, this time with a task failure + complete(stage1Attempt, + Seq((FetchFailed(makeBlockManagerId("hostA"), + shuffleDepOne.shuffleId, 0, 0, "ignored"), null) + ) ++ stage1Successes + ) + } else { + // Run stage 1 + val stage1Attempt = taskSets.last + checkStageId(1, attempt, stage1Attempt) + complete(stage1Attempt, makeCompletions(stage1Attempt, 1)) + + // Fail stage 2 + val stage2Attempt = taskSets.last + checkStageId(2, attempt-Stage.MAX_STAGE_FAILURES/2, stage2Attempt) + complete(stage2Attempt, Seq( + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + } + + // this will (potentially) trigger a resubmission of stage 0, since we've lost some of its + // map output, for the next iteration through the loop + scheduler.resubmitFailedStages() + } + + // Complete all three stages succesfully + val stage0Attempt4 = taskSets.last + checkStageId(0, 4, stage0Attempt4) + complete(stage0Attempt4, makeCompletions(stage0Attempt4, 2)) + + val stage1Attempt4 = taskSets.last + checkStageId(1, 4, stage1Attempt4) + complete(stage1Attempt4, makeCompletions(stage1Attempt4, 1)) + + println(taskSets.mkString(",")) + + val stage2Attempt = taskSets.last + checkStageId(2, Stage.MAX_STAGE_FAILURES/2, stage2Attempt) + complete(stage2Attempt, Seq((Success, 42))) + + // The first success is from the success we append in stage 1, the second is the one we add here + assert(results === Map(0 -> 42)) + } + + /** + * In this test we simulate a job failure where a stage may have many tasks, many of which fail. + * We want to show that many fetch failures inside a single stage do not trigger an abort on + * their own, but only when the stage fails enough times. + */ + test("Multiple task failures in same stage should not abort the stage.") { + setupStageAbortTest(sc) + + val parts = 8 + val shuffleMapRdd = new MyRDD(sc, parts, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, parts, List(shuffleDep)) + submit(reduceRdd, (0 until parts).toArray) + + val stage0Attempt0 = taskSets.last + checkStageId(0, 0, stage0Attempt0) + // Make each task in stage 0 success, then fail all of stage 1 + complete(stage0Attempt0, makeCompletions(stage0Attempt0, parts)) + + val stage1Attempt0 = taskSets.last + checkStageId(1, 0, stage1Attempt0) + val failures = stage1Attempt0.tasks.zipWithIndex.map{ case (task, idx) => + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, idx, "ignored"), null) + }.toSeq + + // Run Stage 1 with all fetchs failing + complete(stage1Attempt0, failures) + + // Resubmit and confirm that now all is well + scheduler.resubmitFailedStages() + + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + + // Confirm job finished succesfully + val stage0Attempt1 = taskSets.last + checkStageId(0, 1, stage0Attempt1) + complete(stage0Attempt1, makeCompletions(stage0Attempt1, 8)) + + val stage1Attempt1 = taskSets.last + checkStageId(1, 1, stage1Attempt1) + complete(stage1Attempt1, stage1Attempt1.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq) + + sc.listenerBus.waitUntilEmpty(1000) + assert(ended === true) + assert(results === (0 until parts).map{idx => idx -> 42}.toMap) + } + + /** + * In this test we demonstrate that only consecutive failures trigger a stage abort. A stage may + * fail multiples, succeed, then fail a few more times (because its run again by downstream + * dependencies). The total number of failed attempts for one stage will go over the limit, + * but that doesn't matter, since they have successes in the middle. + */ + test("Abort should only trigger after consecutive stage failures") { + setupStageAbortTest(sc) + + val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache() + val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)).cache() + val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) + submit(finalRdd, Array(0)) + + for (attempt <- 0 until Stage.MAX_STAGE_FAILURES-1) { + // Make each task in stage 0 success + val stage0Attempt = taskSets.last + checkStageId(0, attempt, stage0Attempt) + + // Run stage 0 + complete(stage0Attempt, makeCompletions(stage0Attempt, 2)) + + // Now we should have a new taskSet, for a new attempt of stage 1. + // We will have one fetch failure for this task set + val stage1Attempt = taskSets.last + checkStageId(1, attempt, stage1Attempt) + + val stage1Successes = + stage1Attempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} + + // Run Stage 1, this time with a task failure + complete(stage1Attempt, + Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null) + ) ++ stage1Successes + ) + + scheduler.resubmitFailedStages() + + // Confirm we have not yet aborted + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + } + + // Rerun stage 0 + val stage0Attempt3 = taskSets.last + checkStageId(0, 3, stage0Attempt3) + complete(stage0Attempt3, makeCompletions(stage0Attempt3, 2)) + + // Now succeed stage 1 and fail stage 2 + val stage1Attempt3 = taskSets.last + checkStageId(1, 3, stage1Attempt3) + // Succeed 1 + complete(stage1Attempt3, makeCompletions(stage1Attempt3, 1)) + + // Fail stage 2 + val stage2Attempt0 = taskSets.last + checkStageId(2, 0, stage2Attempt0) + complete(stage2Attempt0, Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + + scheduler.resubmitFailedStages() + // Rerun stage 0 + val stage0Attempt4 = taskSets.last + checkStageId(0, 4, stage0Attempt4) + complete(stage0Attempt4, makeCompletions(stage0Attempt4, 2)) + + // Now again, fail stage 1 (up to MAX_FAILURES) but confirm that this doesn't trigger an abort + // since we succeeded in between + val stage1attempt4 = taskSets.last + checkStageId(1, 4, stage1attempt4) + val stage1Successes = + stage1attempt4.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} + + // Run Stage 1, this time with a task failure + complete(stage1attempt4, + Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null) + ) ++ stage1Successes + ) + + Thread.sleep(500) + scheduler.resubmitFailedStages() + Thread.sleep(500) --- End diff -- the sleep isn't necessary, right?
--- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org