Github user squito commented on a diff in the pull request:
https://github.com/apache/spark/pull/5636#discussion_r38604678
--- Diff:
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala ---
@@ -473,6 +473,282 @@ class DAGSchedulerSuite
assertDataStructuresEmpty()
}
+
+ // Helper function to validate state when creating tests for task
failures
+ private def checkStageId(stageId: Int, attempt: Int, stageAttempt:
TaskSet) {
+ assert(stageAttempt.stageId === stageId)
+ assert(stageAttempt.stageAttemptId == attempt)
+ }
+
+
+ // Helper functions to extract commonly used code in Fetch Failure test
cases
+ private 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
+ }
+ }
+
+ /**
+ * Common code to get the next stage attempt, confirm it's the one we
expect, and complete it
+ * successfully.
+ *
+ * @param stageId - The current stageId
+ * @param attemptIdx - The current attempt count
+ * @param numShufflePartitions - The number of partitions in the next
stage
+ */
+ private def completeShuffleMapStageSuccessfully(
+ stageId: Int,
+ attemptIdx: Int,
+ numShufflePartitions: Int): Unit = {
+ val stageAttempt = taskSets.last
+ checkStageId(stageId, attemptIdx, stageAttempt)
+ complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map {
+ case (task, idx) =>
+ (Success, makeMapStatus("host" + ('A' + idx).toChar,
numShufflePartitions))
+ }.toSeq)
+ }
+
+ /**
+ * 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
+ */
+ private 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
+ */
+ private 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 where many tasks in the same stage
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("Single fetch failure 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)
+
+ completeShuffleMapStageSuccessfully(0, 0, numShufflePartitions = parts)
+
+ completeNextStageWithFetchFailure(1, 0, shuffleDep)
--- End diff --
wait I think things got a little confused between all the comments from
Kay, Andrew, and me ...
As this stands now, its not a single fetch failure -- there is fetch
failure from every task. I think the options were either (a) move this test to
be first (as you've already done), but keep the name "multiple tasks w/ fetch
failures" or (b) change the *other* tests to only have a single fetch failure
by the refactoring to `completeStageWithFetchFailure`, and keep this one w/
multiple tasks w/ fetch failures.
Maybe the name should actually be "multiple task with fetch failures in a
single stage attempt should not abort the stage"?
---
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]