Github user squito commented on a diff in the pull request: https://github.com/apache/spark/pull/5636#discussion_r36405856 --- Diff: core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala --- @@ -473,6 +473,280 @@ 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 + } + } + + // Helper functions to extract commonly used code in Fetch Failure test cases + /** + * Common code to get the next stage attempt, confirm it's the one we expect, and complete it + * succesfullly. + * + * @param stageId - The current stageId + * @param attemptIdx - The current attempt count + * @param numShufflePartitions - The number of partitions in the next stage + */ + def completeNextShuffleMapSuccesfully(stageId: Int, attemptIdx: Int, + numShufflePartitions: Int): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + complete(stageAttempt, makeCompletions(stageAttempt, numShufflePartitions)) + } + + /** + * Common code to get the next stage attempt, confirm it's the one we expect, and complete it + * with all FetchFailure. + * + * @param stageId - The current stageId + * @param attemptIdx - The current attempt count + * @param shuffleDep - The shuffle dependency of the stage with a fetch failure + */ + def completeNextStageWithFetchFailure(stageId: Int, + attemptIdx: Int, + shuffleDep: ShuffleDependency[_, _, _]): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + + complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map{ case (task, idx) => + (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) + }.toSeq) + } + + /** + * Common code to get the next result stage attempt, confirm it's the one we expect, and + * complete it with a success where we return 42. + * + * @param stageId - The current stageId + * @param attemptIdx - The current attempt count + */ + def completeNextResultStageWithSuccess (stageId: Int, attemptIdx: Int): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage]) + complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq) + } + + /** + * 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 job abort to avoid endless retries. + */ + test("Multiple consecutive stage failures should lead to job 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 + completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 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 + completeNextStageWithFetchFailure(1, attempt, shuffleDep) + + // 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) + jobResult match { + case JobFailed(reason) => + assert(reason.getMessage.contains("ResultStage 1 () has failed the maximum")) + case other => fail(s"expected JobFailed, not $other") + } + } + } + } + + /** + * In this test we simulate a job failure where there are two failures in two different stages. + * Specifically, stage1 fails twice, and then stage2 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 + completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 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 + completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) + } else { + completeNextShuffleMapSuccesfully(1, attempt, numShufflePartitions = 1) + + // Fail stage 2 + completeNextStageWithFetchFailure(2, attempt - Stage.MAX_STAGE_FAILURES / 2, + shuffleDepTwo) + } + + // 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() + } + + completeNextShuffleMapSuccesfully(0, 4, numShufflePartitions = 2) + completeNextShuffleMapSuccesfully(1, 4, numShufflePartitions = 1) + + // Succeed stage2 with a "42" + completeNextResultStageWithSuccess(2, Stage.MAX_STAGE_FAILURES/2) + + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty() + } + + /** + * 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 attempt do not trigger an abort + * on their own, but only when there are enough failing stage attempts. + */ + test("Multiple tasks w/ fetch failures in same stage should not abort the stage.") { --- End diff -- ... in same stage *attempt* should not ...
--- 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