Github user ilganeli commented on a diff in the pull request: https://github.com/apache/spark/pull/5636#discussion_r35695602 --- Diff: core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala --- @@ -473,6 +473,319 @@ 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-1) + } + + /** + * 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.") { + // 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 + } + } + + sc.listenerBus.addListener(new EndListener()) + + 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 <- 1 to Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last + + // Confirm that this is the first attempt for stage 0 + checkStageId(0, attempt, stage0Attempt) + + // Make each task in stage 0 success + val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, 2)) + }.toSeq + + // Run stage 0 + complete(stage0Attempt, completions) + + // 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) { + 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") { + 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)) + + // In the first two iterations, Stage 0 succeeds and stage 1 fails. In the next two iterations, + // stage 0 fails. + for (attempt <- 1 to Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last + + // Confirm that this is the first attempt for stage 0 + checkStageId(0, attempt, stage0Attempt) + + if (attempt < Stage.MAX_STAGE_FAILURES/2) { + // Make each task in stage 0 success + val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, 2)) + }.toSeq + + // Run stage 0 + complete(stage0Attempt, completions) + + // 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 + ) + } else { + val stage0Successes = stage0Attempt.tasks.tail.map { _ => (Success, 42)} + + // Run stage 0 and fail + complete(stage0Attempt, + Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) + ++ stage0Successes --- End diff -- Argh :(
--- 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