From 40aefbedda98828d191ba463725cd1278b0b25ad Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 22 Apr 2015 11:07:23 -0700 Subject: [PATCH 01/45] [SPARK-5945] Added map to track reasons for stage failures and supporting function to check whether to abort a stage when it fails for a single reason more than N times. --- .../apache/spark/scheduler/DAGScheduler.scala | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4a32f8936fb0e..cb77a2f0b4f90 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -96,6 +96,30 @@ class DAGScheduler( // Stages that must be resubmitted due to fetch failures private[scheduler] val failedStages = new HashSet[Stage] + // The maximum number of times to retry a stage before aborting + val maxStageFailures = 5 + + // To avoid cyclical stage failures (see SPARK-5945) we limit the number of times that a stage + // may be retried. However, it only makes sense to limit the number of times that a stage fails + // if it's failing for the same reason every time. Therefore, track why a stage fails as well as + // how many times it has failed. + case class StageFailure(failureReason : String) { + var count = 1 + def fail() = { count += 1 } + def shouldAbort(): Boolean = { count >= maxStageFailures } + + override def equals(other: Any): Boolean = + other match { + case that: StageFailure => that.failureReason.equals(this.failureReason) + case _ => false + } + + override def hashCode: Int = failureReason.hashCode() + } + + // Map to track failure reasons for a given stage (indexed by stage ID) + private[scheduler] val stageFailureReasons = new HashMap[Stage, HashSet[StageFailure]] + private[scheduler] val activeJobs = new HashSet[ActiveJob] /** @@ -460,6 +484,10 @@ class DAGScheduler( logDebug("Removing stage %d from failed set.".format(stageId)) failedStages -= stage } + if (stageFailureReasons.contains(stage)) { + logDebug("Removing stage %d from failure reasons set.".format(stageId)) + stageFailureReasons -= stage + } } // data structures based on StageId stageIdToStage -= stageId @@ -940,6 +968,29 @@ class DAGScheduler( } } + /** + * Check whether we should abort the failedStage due to multiple failures for the same reason. + * This method updates the running count of failures for a particular stage and returns + * true if the number of failures for any single reason exceeds the allowable number + * of failures. + * @return An Option that contains the failure reason that caused the abort + */ + def shouldAbortStage(failedStage: Stage, failureReason: String): Option[String] = { + if (!stageFailureReasons.contains(failedStage)) + stageFailureReasons.put(failedStage, new HashSet[StageFailure]()) + + val failures = stageFailureReasons.get(failedStage).get + val failure = StageFailure(failureReason) + failures.find(s => s.equals(failure)) match { + case Some(f) => f.fail() + case None => failures.add(failure) + } + failures.find(_.shouldAbort()) match { + case Some(f) => Some(f.failureReason) + case None => None + } + } + /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -1083,8 +1134,13 @@ class DAGScheduler( markStageAsFinished(failedStage, Some(failureMessage)) } + val shouldAbort = shouldAbortStage(failedStage, failureMessage) if (disallowStageRetryForTest) { abortStage(failedStage, "Fetch failure will not retry stage due to testing config") + } else if (shouldAbort.isDefined) { + abortStage(failedStage, s"Fetch failure - aborting stage. Stage ${failedStage.name} " + + s"has failed the maximum allowable number of times: ${maxStageFailures}. " + + s"Failure reason: ${shouldAbort.get}") } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. From f8744be6ef59a3af2c1cd2060e14617d70825ac5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 22 Apr 2015 11:23:34 -0700 Subject: [PATCH 02/45] Fixed method scoping error --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cb77a2f0b4f90..78076c6128082 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -975,9 +975,11 @@ class DAGScheduler( * of failures. * @return An Option that contains the failure reason that caused the abort */ + private[scheduler] def shouldAbortStage(failedStage: Stage, failureReason: String): Option[String] = { - if (!stageFailureReasons.contains(failedStage)) + if (!stageFailureReasons.contains(failedStage)) { stageFailureReasons.put(failedStage, new HashSet[StageFailure]()) + } val failures = stageFailureReasons.get(failedStage).get val failure = StageFailure(failureReason) From 8fe31e05fbe28b70ea4daee3be654a3f02686abc Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 22 Apr 2015 11:33:12 -0700 Subject: [PATCH 03/45] Made StageFailure private to spark scheduler --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 78076c6128082..7099bdfc65584 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -103,7 +103,7 @@ class DAGScheduler( // may be retried. However, it only makes sense to limit the number of times that a stage fails // if it's failing for the same reason every time. Therefore, track why a stage fails as well as // how many times it has failed. - case class StageFailure(failureReason : String) { + private[scheduler] case class StageFailure(failureReason : String) { var count = 1 def fail() = { count += 1 } def shouldAbort(): Boolean = { count >= maxStageFailures } From e0f8b55b4f0511aa00cae7e8a56b557c8c0ef463 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 22 Apr 2015 11:41:48 -0700 Subject: [PATCH 04/45] Made fail() method public --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7099bdfc65584..370bcc50fcfd0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -105,7 +105,7 @@ class DAGScheduler( // how many times it has failed. private[scheduler] case class StageFailure(failureReason : String) { var count = 1 - def fail() = { count += 1 } + def fail(): Unit = { count += 1 } def shouldAbort(): Boolean = { count >= maxStageFailures } override def equals(other: Any): Boolean = From 729b7efd63bbe6265bc0689f91ac9c2c454dec4b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 22 Apr 2015 11:44:27 -0700 Subject: [PATCH 05/45] Added config option for stageFailure count and documentation --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- docs/configuration.md | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 370bcc50fcfd0..7cf89d25dc70b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -97,7 +97,7 @@ class DAGScheduler( private[scheduler] val failedStages = new HashSet[Stage] // The maximum number of times to retry a stage before aborting - val maxStageFailures = 5 + val maxStageFailures = sc.conf.getInt("spark.stage.maxFailures", 5) // To avoid cyclical stage failures (see SPARK-5945) we limit the number of times that a stage // may be retried. However, it only makes sense to limit the number of times that a stage fails diff --git a/docs/configuration.md b/docs/configuration.md index d587b91124cb8..f01765662f63c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1153,6 +1153,14 @@ Apart from these, the following properties are also available, and may be useful Should be greater than or equal to 1. Number of allowed retries = this value - 1. + + spark.stage.maxFailures + 5 + + Number of individual stage failures before aborting the stage and not retrying it. + Should be greater than or equal to 1. Number of allowed retries = this value - 1. + + #### Dynamic Allocation From d5fa622601b14df6d58582a06d7a05f73f9f1fe7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 23 Apr 2015 18:24:36 -0700 Subject: [PATCH 06/45] Moved failure tracking to Stage class. Added clear of failre count upon Stage success --- .../apache/spark/scheduler/DAGScheduler.scala | 64 ++----------------- .../org/apache/spark/scheduler/Stage.scala | 20 ++++++ 2 files changed, 27 insertions(+), 57 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7cf89d25dc70b..e757bb75b5dc2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -95,30 +95,6 @@ class DAGScheduler( // Stages that must be resubmitted due to fetch failures private[scheduler] val failedStages = new HashSet[Stage] - - // The maximum number of times to retry a stage before aborting - val maxStageFailures = sc.conf.getInt("spark.stage.maxFailures", 5) - - // To avoid cyclical stage failures (see SPARK-5945) we limit the number of times that a stage - // may be retried. However, it only makes sense to limit the number of times that a stage fails - // if it's failing for the same reason every time. Therefore, track why a stage fails as well as - // how many times it has failed. - private[scheduler] case class StageFailure(failureReason : String) { - var count = 1 - def fail(): Unit = { count += 1 } - def shouldAbort(): Boolean = { count >= maxStageFailures } - - override def equals(other: Any): Boolean = - other match { - case that: StageFailure => that.failureReason.equals(this.failureReason) - case _ => false - } - - override def hashCode: Int = failureReason.hashCode() - } - - // Map to track failure reasons for a given stage (indexed by stage ID) - private[scheduler] val stageFailureReasons = new HashMap[Stage, HashSet[StageFailure]] private[scheduler] val activeJobs = new HashSet[ActiveJob] @@ -484,10 +460,6 @@ class DAGScheduler( logDebug("Removing stage %d from failed set.".format(stageId)) failedStages -= stage } - if (stageFailureReasons.contains(stage)) { - logDebug("Removing stage %d from failure reasons set.".format(stageId)) - stageFailureReasons -= stage - } } // data structures based on StageId stageIdToStage -= stageId @@ -968,31 +940,6 @@ class DAGScheduler( } } - /** - * Check whether we should abort the failedStage due to multiple failures for the same reason. - * This method updates the running count of failures for a particular stage and returns - * true if the number of failures for any single reason exceeds the allowable number - * of failures. - * @return An Option that contains the failure reason that caused the abort - */ - private[scheduler] - def shouldAbortStage(failedStage: Stage, failureReason: String): Option[String] = { - if (!stageFailureReasons.contains(failedStage)) { - stageFailureReasons.put(failedStage, new HashSet[StageFailure]()) - } - - val failures = stageFailureReasons.get(failedStage).get - val failure = StageFailure(failureReason) - failures.find(s => s.equals(failure)) match { - case Some(f) => f.fail() - case None => failures.add(failure) - } - failures.find(_.shouldAbort()) match { - case Some(f) => Some(f.failureReason) - case None => None - } - } - /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -1021,6 +968,10 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) event.reason match { case Success => + // Clear failure count for this stage, now that it's succeeded. This ensures that even if + // subsequent stages fail, triggering a recompute of this stage, we abort because of + // those failures. + stage.clearFailures() listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, event.reason, event.taskInfo, event.taskMetrics)) stage.pendingTasks -= task @@ -1136,13 +1087,12 @@ class DAGScheduler( markStageAsFinished(failedStage, Some(failureMessage)) } - val shouldAbort = shouldAbortStage(failedStage, failureMessage) if (disallowStageRetryForTest) { abortStage(failedStage, "Fetch failure will not retry stage due to testing config") - } else if (shouldAbort.isDefined) { + } else if (failedStage.failAndShouldAbort()) { abortStage(failedStage, s"Fetch failure - aborting stage. Stage ${failedStage.name} " + - s"has failed the maximum allowable number of times: ${maxStageFailures}. " + - s"Failure reason: ${shouldAbort.get}") + s"has failed the maximum allowable number of times: ${failedStage.maxStageFailures}. " + + s"Failure reason: ${failureMessage}") } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 5d0ddb8377c33..e44f5427d13a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -70,6 +70,26 @@ private[spark] abstract class Stage( /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ var latestInfo: StageInfo = StageInfo.fromStage(this) + // The maximum number of times to retry a stage before aborting + final val maxStageFailures = 4 + + // To avoid cyclical stage failures (see SPARK-5945) we limit the number of times that a stage + // may be retried. + private var failCount = 0 + private[scheduler] def fail() : Unit = { failCount += 1 } + private[scheduler] def shouldAbort(): Boolean = { failCount > maxStageFailures } + private[scheduler] def clearFailures() : Unit = { failCount = 0 } + + /** + * Check whether we should abort the failedStage due to multiple failures. + * This method updates the running count of failures for a particular stage and returns + * true if the number of failures exceeds the allowable number of failures. + */ + private[scheduler] def failAndShouldAbort(): Boolean = { + fail() + shouldAbort() + } + /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { val id = nextAttemptId From 0335b967b4b1a91782b5a608220c9c3eeb0bf8e1 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 23 Apr 2015 18:32:23 -0700 Subject: [PATCH 07/45] Removed stale documentation and fixed some erroneous spacing --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- docs/configuration.md | 8 -------- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e757bb75b5dc2..c8cecc32bd1d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -95,7 +95,7 @@ class DAGScheduler( // Stages that must be resubmitted due to fetch failures private[scheduler] val failedStages = new HashSet[Stage] - + private[scheduler] val activeJobs = new HashSet[ActiveJob] /** diff --git a/docs/configuration.md b/docs/configuration.md index f01765662f63c..d587b91124cb8 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1153,14 +1153,6 @@ Apart from these, the following properties are also available, and may be useful Should be greater than or equal to 1. Number of allowed retries = this value - 1. - - spark.stage.maxFailures - 5 - - Number of individual stage failures before aborting the stage and not retrying it. - Should be greater than or equal to 1. Number of allowed retries = this value - 1. - - #### Dynamic Allocation From 2b919401f96bab041bd758fb9575dbce0d7d5c2c Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 27 Apr 2015 17:16:30 -0700 Subject: [PATCH 08/45] Added test case for stage abort after N failures --- .../apache/spark/scheduler/DAGScheduler.scala | 9 +++--- .../org/apache/spark/scheduler/Stage.scala | 4 +-- .../spark/scheduler/DAGSchedulerSuite.scala | 29 +++++++++++++++++++ 3 files changed, 36 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c8cecc32bd1d0..34fbda278cbb4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -968,10 +968,6 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) event.reason match { case Success => - // Clear failure count for this stage, now that it's succeeded. This ensures that even if - // subsequent stages fail, triggering a recompute of this stage, we abort because of - // those failures. - stage.clearFailures() listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, event.reason, event.taskInfo, event.taskMetrics)) stage.pendingTasks -= task @@ -988,6 +984,11 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { + // Clear failure count for this stage, now that it's succeeded. + // This ensures that even if subsequent stages fail, triggering + // a recompute of this stage, we abort because of those failures. + stage.clearFailures() + markStageAsFinished(resultStage) cleanupStateForJobAndIndependentStages(job) listenerBus.post( diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index e44f5427d13a1..a621226b47bfe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -71,13 +71,13 @@ private[spark] abstract class Stage( var latestInfo: StageInfo = StageInfo.fromStage(this) // The maximum number of times to retry a stage before aborting - final val maxStageFailures = 4 + val maxStageFailures = 4 // To avoid cyclical stage failures (see SPARK-5945) we limit the number of times that a stage // may be retried. private var failCount = 0 private[scheduler] def fail() : Unit = { failCount += 1 } - private[scheduler] def shouldAbort(): Boolean = { failCount > maxStageFailures } + private[scheduler] def shouldAbort(): Boolean = { failCount >= maxStageFailures } private[scheduler] def clearFailures() : Unit = { failCount = 0 } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3c52a8c4460c6..1e4a29ad0737e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -475,7 +475,36 @@ class DAGSchedulerSuite assert(results === Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() } + + test("Test taskAbort after multiple stage failures.") { + 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)) + scheduler.resubmitFailedStages() + + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + + // Create stage object to get maxStageFailures + val stage = new ResultStage(0, reduceRdd, 0, null, 0, new CallSite("blah","blah")) + for (x <- 1 to stage.maxStageFailures) { + // the 2nd ResultTask failed + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + scheduler.resubmitFailedStages() + if (x < stage.maxStageFailures) { + assert(!scheduler.runningStages.isEmpty) + } else { + assertDataStructuresEmpty() + } + } + } + test("trivial shuffle with multiple fetch failures") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 914b2cb4626b59bb4ce79d9ce303ffaec0a2d159 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 27 Apr 2015 17:17:30 -0700 Subject: [PATCH 09/45] Nit --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1e4a29ad0737e..cbd49230f3de6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -498,7 +498,7 @@ class DAGSchedulerSuite scheduler.resubmitFailedStages() if (x < stage.maxStageFailures) { - assert(!scheduler.runningStages.isEmpty) + assert(scheduler.runningStages.nonEmpty) } else { assertDataStructuresEmpty() } From 77555b9ac06b27cd879eef2a44b753f42a0b330d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Apr 2015 14:34:44 -0700 Subject: [PATCH 10/45] Added test that also validates that the listenerBus sees the JobFailed error --- .../apache/spark/scheduler/DAGScheduler.scala | 12 +++++----- .../org/apache/spark/scheduler/Stage.scala | 8 +++---- .../spark/scheduler/DAGSchedulerSuite.scala | 22 +++++++++++++++++++ 3 files changed, 33 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 34fbda278cbb4..9c85aa2ee1ee7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -984,11 +984,6 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - // Clear failure count for this stage, now that it's succeeded. - // This ensures that even if subsequent stages fail, triggering - // a recompute of this stage, we abort because of those failures. - stage.clearFailures() - markStageAsFinished(resultStage) cleanupStateForJobAndIndependentStages(job) listenerBus.post( @@ -1216,10 +1211,17 @@ class DAGScheduler( if (errorMessage.isEmpty) { logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stage.latestInfo.completionTime = Some(clock.getTimeMillis()) + + // Clear failure count for this stage, now that it's succeeded. + // We only limit consecutive failures of stage attempts, such that if this stage is a + // dependency for downstream stages, e.g. in a long-running streaming app, we don't + // fail because of failures of this stage, but rather the failed downstage components. + stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) } + outputCommitCoordinator.stageEnd(stage.id) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) runningStages -= stage diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index a621226b47bfe..e0ee6bcdf1cba 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -70,11 +70,11 @@ private[spark] abstract class Stage( /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ var latestInfo: StageInfo = StageInfo.fromStage(this) - // The maximum number of times to retry a stage before aborting - val maxStageFailures = 4 + // The maximum number of times to retry a stage before aborting + private[scheduler] val maxStageFailures = 4 - // To avoid cyclical stage failures (see SPARK-5945) we limit the number of times that a stage - // may be retried. + // Spark is resilient to executors dying by retrying stages on FetchFailures. This counter ensures + // that we don't retry stages indefinitely by aborting the stage if it fails too many times. private var failCount = 0 private[scheduler] def fail() : Unit = { failCount += 1 } private[scheduler] def shouldAbort(): Boolean = { failCount >= maxStageFailures } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index cbd49230f3de6..9b43256fb6595 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -477,11 +477,26 @@ class DAGSchedulerSuite } test("Test taskAbort after multiple stage failures.") { + // 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)) + sparkListener.failedStages.clear() scheduler.resubmitFailedStages() complete(taskSets(0), Seq( @@ -499,8 +514,15 @@ class DAGSchedulerSuite scheduler.resubmitFailedStages() if (x < stage.maxStageFailures) { assert(scheduler.runningStages.nonEmpty) + assert(!ended) + assert(!jobResult.isInstanceOf[JobFailed]) } else { assertDataStructuresEmpty() + // This should now contain the failed stage + + sc.listenerBus.waitUntilEmpty(1000) + assert(ended) + assert(jobResult.isInstanceOf[JobFailed]) } } } From 1243b656ece26a53f33cbe04e472588310499458 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Apr 2015 14:50:47 -0700 Subject: [PATCH 11/45] updated to fix last few items --- .../main/scala/org/apache/spark/scheduler/Stage.scala | 10 ++++++---- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 11 +++-------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index e0ee6bcdf1cba..63ed24d886937 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -69,15 +69,12 @@ private[spark] abstract class Stage( /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ var latestInfo: StageInfo = StageInfo.fromStage(this) - - // The maximum number of times to retry a stage before aborting - private[scheduler] val maxStageFailures = 4 // Spark is resilient to executors dying by retrying stages on FetchFailures. This counter ensures // that we don't retry stages indefinitely by aborting the stage if it fails too many times. private var failCount = 0 private[scheduler] def fail() : Unit = { failCount += 1 } - private[scheduler] def shouldAbort(): Boolean = { failCount >= maxStageFailures } + private[scheduler] def shouldAbort(): Boolean = { failCount >= Stage.maxStageFailures } private[scheduler] def clearFailures() : Unit = { failCount = 0 } /** @@ -105,3 +102,8 @@ private[spark] abstract class Stage( case _ => false } } + +private[spark] object Stage { + // The maximum number of times to retry a stage before aborting + private[scheduler] val maxStageFailures = 4 +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9b43256fb6595..62b8b4181dd4b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -496,30 +496,25 @@ class DAGSchedulerSuite val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - sparkListener.failedStages.clear() - scheduler.resubmitFailedStages() complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - // Create stage object to get maxStageFailures - val stage = new ResultStage(0, reduceRdd, 0, null, 0, new CallSite("blah","blah")) - for (x <- 1 to stage.maxStageFailures) { + for (x <- 1 to Stage.maxStageFailures) { // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() - if (x < stage.maxStageFailures) { + if (x < Stage.maxStageFailures) { assert(scheduler.runningStages.nonEmpty) assert(!ended) assert(!jobResult.isInstanceOf[JobFailed]) } else { + // Stage has been aborted and removed from running stages assertDataStructuresEmpty() - // This should now contain the failed stage - sc.listenerBus.waitUntilEmpty(1000) assert(ended) assert(jobResult.isInstanceOf[JobFailed]) From 9052e392d32cf03bb572b35cbef0e4dca59ee06c Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Apr 2015 15:03:09 -0700 Subject: [PATCH 12/45] Removed extraneous assert --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 62b8b4181dd4b..9295a11e145cf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -511,7 +511,6 @@ class DAGSchedulerSuite if (x < Stage.maxStageFailures) { assert(scheduler.runningStages.nonEmpty) assert(!ended) - assert(!jobResult.isInstanceOf[JobFailed]) } else { // Stage has been aborted and removed from running stages assertDataStructuresEmpty() From 673fcb2530185bd42e6eab52bb18dabdcab478c3 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Apr 2015 15:04:56 -0700 Subject: [PATCH 13/45] Fixed a couple minor issues. Removed unecessary asserT --- core/src/main/scala/org/apache/spark/scheduler/Stage.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 63ed24d886937..b601966c2dd04 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -82,10 +82,7 @@ private[spark] abstract class Stage( * This method updates the running count of failures for a particular stage and returns * true if the number of failures exceeds the allowable number of failures. */ - private[scheduler] def failAndShouldAbort(): Boolean = { - fail() - shouldAbort() - } + private[scheduler] def failAndShouldAbort(): Boolean = { fail(); shouldAbort() } /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { From e26ae6edda30a139a6c6354c54573e7f15afa446 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Apr 2015 15:06:47 -0700 Subject: [PATCH 14/45] Fixed a couple minor issues. Removed unecessary asserT --- core/src/main/scala/org/apache/spark/scheduler/Stage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index b601966c2dd04..0848d6262e48a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -103,4 +103,4 @@ private[spark] abstract class Stage( private[spark] object Stage { // The maximum number of times to retry a stage before aborting private[scheduler] val maxStageFailures = 4 -} \ No newline at end of file +} From 75952ea04d5ed6e54b5bae1aee4a7c377db414cc Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 30 Apr 2015 15:13:25 -0700 Subject: [PATCH 15/45] Fixed missing maxStageFailures --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9c85aa2ee1ee7..de319f2dedb75 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1087,7 +1087,7 @@ class DAGScheduler( abortStage(failedStage, "Fetch failure will not retry stage due to testing config") } else if (failedStage.failAndShouldAbort()) { abortStage(failedStage, s"Fetch failure - aborting stage. Stage ${failedStage.name} " + - s"has failed the maximum allowable number of times: ${failedStage.maxStageFailures}. " + + s"has failed the maximum allowable number of times: ${Stage.maxStageFailures}. " + s"Failure reason: ${failureMessage}") } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because From bc88aa136021a6ae4bc1763005c357a718a32d89 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 4 May 2015 14:22:12 -0700 Subject: [PATCH 16/45] Updated to add test cases for multiple task failures within a Stage. Updated to only increment fetch failures once per attempt ID for a stage. --- .../apache/spark/scheduler/DAGScheduler.scala | 8 +- .../org/apache/spark/scheduler/Stage.scala | 41 +++++-- .../spark/scheduler/DAGSchedulerSuite.scala | 102 +++++++++++++++++- 3 files changed, 136 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index de319f2dedb75..37bf46fc712c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1087,7 +1087,7 @@ class DAGScheduler( abortStage(failedStage, "Fetch failure will not retry stage due to testing config") } else if (failedStage.failAndShouldAbort()) { abortStage(failedStage, s"Fetch failure - aborting stage. Stage ${failedStage.name} " + - s"has failed the maximum allowable number of times: ${Stage.maxStageFailures}. " + + s"has failed the maximum allowable number of times: ${Stage.MAX_STAGE_FAILURES}. " + s"Failure reason: ${failureMessage}") } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because @@ -1213,9 +1213,9 @@ class DAGScheduler( stage.latestInfo.completionTime = Some(clock.getTimeMillis()) // Clear failure count for this stage, now that it's succeeded. - // We only limit consecutive failures of stage attempts, such that if this stage is a - // dependency for downstream stages, e.g. in a long-running streaming app, we don't - // fail because of failures of this stage, but rather the failed downstage components. + // We only limit consecutive failures of stage attempts,so that if a stage is + // re-used many times in a long-running job, unrelated failures that are spaced out in time + // don't eventually cause the stage to be aborted." stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 0848d6262e48a..795a3fa227706 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import scala.collection.mutable import scala.collection.mutable.HashSet import org.apache.spark._ @@ -69,20 +70,44 @@ private[spark] abstract class Stage( /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ var latestInfo: StageInfo = StageInfo.fromStage(this) + + /** + * Spark is resilient to executors dying by retrying stages on FetchFailures. Here, we keep track + * of the number of stage failures to prevent endless stage retries. However, because + * FetchFailures may cause multiple tasks to retry a Stage in parallel, we cannot count stage + * failures alone since the same stage may be attempted multiple times simultaneously. + * We deal with this by tracking the number of failures per attemptId. + */ + private val failedStageAttemptIds = new mutable.HashMap[Int, HashSet[Int]] + private var failedStageCount = 0 - // Spark is resilient to executors dying by retrying stages on FetchFailures. This counter ensures - // that we don't retry stages indefinitely by aborting the stage if it fails too many times. - private var failCount = 0 - private[scheduler] def fail() : Unit = { failCount += 1 } - private[scheduler] def shouldAbort(): Boolean = { failCount >= Stage.maxStageFailures } - private[scheduler] def clearFailures() : Unit = { failCount = 0 } + private[scheduler] def clearFailures() : Unit = { + failedStageAttemptIds.clear() + failedStageCount = 0 + } /** * Check whether we should abort the failedStage due to multiple failures. * This method updates the running count of failures for a particular stage and returns * true if the number of failures exceeds the allowable number of failures. */ - private[scheduler] def failAndShouldAbort(): Boolean = { fail(); shouldAbort() } + private[scheduler] def failAndShouldAbort(): Boolean = { + // We increment the failure count on the first attempt for a particular Stage + if (latestInfo.attemptId == 0) + { + failedStageCount += 1 + } + + val concurrentFailures = failedStageAttemptIds + .getOrElseUpdate(failedStageCount, new HashSet[Int]()) + + concurrentFailures.add(latestInfo.attemptId) + + // Check for multiple FetchFailures in a Stage and for the stage failing repeatedly following + // resubmissions. + failedStageCount >= Stage.MAX_STAGE_FAILURES || + concurrentFailures.size >= Stage.MAX_STAGE_FAILURES + } /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { @@ -102,5 +127,5 @@ private[spark] abstract class Stage( private[spark] object Stage { // The maximum number of times to retry a stage before aborting - private[scheduler] val maxStageFailures = 4 + private[scheduler] val MAX_STAGE_FAILURES = 4 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9295a11e145cf..305e7070b2379 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -476,7 +476,7 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } - test("Test taskAbort after multiple stage failures.") { + test("Multiple consecutive stage failures should lead to stage 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. @@ -501,14 +501,14 @@ class DAGSchedulerSuite (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - for (x <- 1 to Stage.maxStageFailures) { + for (x <- 1 to Stage.MAX_STAGE_FAILURES) { // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() - if (x < Stage.maxStageFailures) { + if (x < Stage.MAX_STAGE_FAILURES) { assert(scheduler.runningStages.nonEmpty) assert(!ended) } else { @@ -520,6 +520,102 @@ class DAGSchedulerSuite } } } + + + test("Multiple consecutive Fetch failures in a stage triggers an abort.") { + // 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, 8, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 8, List(shuffleDep)) + submit(reduceRdd, Array(0, 1, 2, 3, 4, 5, 6, 7)) + + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored1"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored2"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored3"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored4"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored5"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored6"), null))) + + scheduler.resubmitFailedStages() + assertDataStructuresEmpty() + sc.listenerBus.waitUntilEmpty(1000) + assert(ended) + assert(jobResult.isInstanceOf[JobFailed]) + } + + test("Multiple consecutive task failures (not FetchFailures) in a stage should not " + + "trigger an abort.") { + // 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, 8, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 8, List(shuffleDep)) + submit(reduceRdd, Array(0, 1, 2, 3, 4, 5, 6, 7)) + + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored1"), null), + (ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failB", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failC", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failD", null, "This is a stack.", None), null), + (Success, 43))) + + scheduler.resubmitFailedStages() + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + } test("trivial shuffle with multiple fetch failures") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) From 7ff8b21d314785bfbb806d5e0e11086f92961a34 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 4 May 2015 14:38:22 -0700 Subject: [PATCH 17/45] Typo fix --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 305e7070b2379..933b8b87fbc7b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -522,7 +522,7 @@ class DAGSchedulerSuite } - test("Multiple consecutive Fetch failures in a stage triggers an abort.") { + test("Multiple consecutive Fetch failures in a stage should trigger an abort.") { // 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. From fe647d0497c69fcef7660d3eb1d724e7505e883d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 23 Jul 2015 16:51:36 -0500 Subject: [PATCH 18/45] Style --- .../apache/spark/scheduler/DAGScheduler.scala | 8 +++--- .../org/apache/spark/scheduler/Stage.scala | 8 +++--- .../spark/scheduler/DAGSchedulerSuite.scala | 28 +++++++++---------- 3 files changed, 22 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 706f07db5e83c..ca62e6d8f9faf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1194,16 +1194,16 @@ class DAGScheduler( logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stage.latestInfo.completionTime = Some(clock.getTimeMillis()) - // Clear failure count for this stage, now that it's succeeded. - // We only limit consecutive failures of stage attempts,so that if a stage is - // re-used many times in a long-running job, unrelated failures that are spaced out in time + // Clear failure count for this stage, now that it's succeeded. + // We only limit consecutive failures of stage attempts,so that if a stage is + // re-used many times in a long-running job, unrelated failures that are spaced out in time // don't eventually cause the stage to be aborted." stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) } - + outputCommitCoordinator.stageEnd(stage.id) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) runningStages -= stage diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 34bbb16118060..f90e9f018a76b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -81,14 +81,14 @@ private[spark] abstract class Stage( * of the number of stage failures to prevent endless stage retries. */ private var failedStageCount = 0 - + private[scheduler] def clearFailures() : Unit = { failedStageCount = 0 } /** * Check whether we should abort the failedStage due to multiple failures. - * This method updates the running count of failures for a particular stage and returns + * This method updates the running count of failures for a particular stage and returns * true if the number of failures exceeds the allowable number of failures. */ private[scheduler] def failAndShouldAbort(): Boolean = { @@ -97,7 +97,7 @@ private[spark] abstract class Stage( { failedStageCount += 1 } - + // Check for multiple FetchFailures in a Stage and for the stage failing repeatedly following // resubmissions. failedStageCount >= Stage.MAX_STAGE_FAILURES @@ -120,6 +120,6 @@ private[spark] abstract class Stage( } private[spark] object Stage { - // The maximum number of times to retry a stage before aborting + // The maximum number of times to retry a stage before aborting private[scheduler] val MAX_STAGE_FAILURES = 4 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index abf61925a061a..053ec4d6f583b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -472,11 +472,11 @@ class DAGSchedulerSuite assert(results === Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() } - + test("Multiple consecutive stage failures should lead to stage being aborted.") { - // Create a new Listener to confirm that the listenerBus sees the JobEnd message + // 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. + // so this will propagate up to the user. var ended = false var jobResult : JobResult = null class EndListener extends SparkListener { @@ -487,17 +487,17 @@ class DAGSchedulerSuite } 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)) - + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - + for (x <- 1 to Stage.MAX_STAGE_FAILURES) { // the 2nd ResultTask failed complete(taskSets(1), Seq( @@ -520,9 +520,9 @@ class DAGSchedulerSuite test("Multiple consecutive Fetch failures in a stage should trigger an abort.") { - // Create a new Listener to confirm that the listenerBus sees the JobEnd message + // 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. + // so this will propagate up to the user. var ended = false var jobResult : JobResult = null class EndListener extends SparkListener { @@ -539,7 +539,7 @@ class DAGSchedulerSuite val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 8, List(shuffleDep)) submit(reduceRdd, Array(0, 1, 2, 3, 4, 5, 6, 7)) - + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostA", 1)), @@ -569,9 +569,9 @@ class DAGSchedulerSuite test("Multiple consecutive task failures (not FetchFailures) in a stage should not " + "trigger an abort.") { - // Create a new Listener to confirm that the listenerBus sees the JobEnd message + // 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. + // so this will propagate up to the user. var ended = false var jobResult : JobResult = null class EndListener extends SparkListener { @@ -601,19 +601,19 @@ class DAGSchedulerSuite complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored1"), null), (ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), null), (ExceptionFailure("fakeExcept", "failB", null, "This is a stack.", None), null), (ExceptionFailure("fakeExcept", "failC", null, "This is a stack.", None), null), (ExceptionFailure("fakeExcept", "failD", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failE", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failF", null, "This is a stack.", None), null), (Success, 43))) scheduler.resubmitFailedStages() assert(scheduler.runningStages.nonEmpty) assert(!ended) } - + test("trivial shuffle with multiple fetch failures") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From ddfe46c36c5755f77ec12925fb674e6d9a119c56 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 23 Jul 2015 17:40:52 -0500 Subject: [PATCH 19/45] restoring lost files --- .../spark/sql/execution/pythonUDFs.scala | 382 +++++++++++ .../org/apache/spark/sql/hive/hiveUDFs.scala | 592 ++++++++++++++++++ 2 files changed, 974 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala new file mode 100644 index 0000000000000..e6e27a87c7151 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala @@ -0,0 +1,382 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.execution + +import java.io.OutputStream +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + +import net.razorvine.pickle._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.python.{PythonBroadcast, PythonRDD, SerDeUtil} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.{Accumulator, Logging => SparkLogging} + +/** + * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. + */ +private[spark] case class PythonUDF( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + pythonVer: String, + broadcastVars: JList[Broadcast[PythonBroadcast]], + accumulator: Accumulator[JList[Array[Byte]]], + dataType: DataType, + children: Seq[Expression]) extends Expression with Unevaluable with SparkLogging { + + override def toString: String = s"PythonUDF#$name(${children.mkString(",")})" + + override def nullable: Boolean = true +} + +/** + * Extracts PythonUDFs from operators, rewriting the query plan so that the UDF can be evaluated + * alone in a batch. + * + * This has the limitation that the input to the Python UDF is not allowed include attributes from + * multiple child operators. + */ +private[spark] object ExtractPythonUDFs extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Skip EvaluatePython nodes. + case plan: EvaluatePython => plan + + case plan: LogicalPlan if plan.resolved => + // Extract any PythonUDFs from the current operator. + val udfs = plan.expressions.flatMap(_.collect { case udf: PythonUDF => udf }) + if (udfs.isEmpty) { + // If there aren't any, we are done. + plan + } else { + // Pick the UDF we are going to evaluate (TODO: Support evaluating multiple UDFs at a time) + // If there is more than one, we will add another evaluation operator in a subsequent pass. + udfs.find(_.resolved) match { + case Some(udf) => + var evaluation: EvaluatePython = null + + // Rewrite the child that has the input required for the UDF + val newChildren = plan.children.map { child => + // Check to make sure that the UDF can be evaluated with only the input of this child. + // Other cases are disallowed as they are ambiguous or would require a cartesian + // product. + if (udf.references.subsetOf(child.outputSet)) { + evaluation = EvaluatePython(udf, child) + evaluation + } else if (udf.references.intersect(child.outputSet).nonEmpty) { + sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") + } else { + child + } + } + + assert(evaluation != null, "Unable to evaluate PythonUDF. Missing input attributes.") + + // Trim away the new UDF value if it was only used for filtering or something. + logical.Project( + plan.output, + plan.transformExpressions { + case p: PythonUDF if p.fastEquals(udf) => evaluation.resultAttribute + }.withNewChildren(newChildren)) + + case None => + // If there is no Python UDF that is resolved, skip this round. + plan + } + } + } +} + +object EvaluatePython { + def apply(udf: PythonUDF, child: LogicalPlan): EvaluatePython = + new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) + + /** + * Helper for converting from Catalyst type to java type suitable for Pyrolite. + */ + def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + case (null, _) => null + + case (row: InternalRow, struct: StructType) => + val values = new Array[Any](row.size) + var i = 0 + while (i < row.size) { + values(i) = toJava(row(i), struct.fields(i).dataType) + i += 1 + } + new GenericInternalRowWithSchema(values, struct) + + case (seq: Seq[Any], array: ArrayType) => + seq.map(x => toJava(x, array.elementType)).asJava + + case (obj: Map[_, _], mt: MapType) => obj.map { + case (k, v) => (toJava(k, mt.keyType), toJava(v, mt.valueType)) + }.asJava + + case (ud, udt: UserDefinedType[_]) => toJava(ud, udt.sqlType) + + case (d: Decimal, _) => d.toJavaBigDecimal + + case (s: UTF8String, StringType) => s.toString + + case (other, _) => other + } + + /** + * Converts `obj` to the type specified by the data type, or returns null if the type of obj is + * unexpected. Because Python doesn't enforce the type. + */ + def fromJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + case (null, _) => null + + case (c: Boolean, BooleanType) => c + + case (c: Int, ByteType) => c.toByte + case (c: Long, ByteType) => c.toByte + + case (c: Int, ShortType) => c.toShort + case (c: Long, ShortType) => c.toShort + + case (c: Int, IntegerType) => c + case (c: Long, IntegerType) => c.toInt + + case (c: Int, LongType) => c.toLong + case (c: Long, LongType) => c + + case (c: Double, FloatType) => c.toFloat + + case (c: Double, DoubleType) => c + + case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c) + + case (c: Int, DateType) => c + + case (c: Long, TimestampType) => c + + case (c: String, StringType) => UTF8String.fromString(c) + case (c, StringType) => + // If we get here, c is not a string. Call toString on it. + UTF8String.fromString(c.toString) + + case (c: String, BinaryType) => c.getBytes("utf-8") + case (c, BinaryType) if c.getClass.isArray && c.getClass.getComponentType.getName == "byte" => c + + case (c: java.util.List[_], ArrayType(elementType, _)) => + c.map { e => fromJava(e, elementType)}.toSeq + + case (c, ArrayType(elementType, _)) if c.getClass.isArray => + c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType)).toSeq + + case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { + case (key, value) => (fromJava(key, keyType), fromJava(value, valueType)) + }.toMap + + case (c, StructType(fields)) if c.getClass.isArray => + new GenericInternalRow(c.asInstanceOf[Array[_]].zip(fields).map { + case (e, f) => fromJava(e, f.dataType) + }) + + case (_, udt: UserDefinedType[_]) => fromJava(obj, udt.sqlType) + + // all other unexpected type should be null, or we will have runtime exception + // TODO(davies): we could improve this by try to cast the object to expected type + case (c, _) => null + } + + + private val module = "pyspark.sql.types" + + /** + * Pickler for StructType + */ + private class StructTypePickler extends IObjectPickler { + + private val cls = classOf[StructType] + + def register(): Unit = { + Pickler.registerCustomPickler(cls, this) + } + + def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + out.write(Opcodes.GLOBAL) + out.write((module + "\n" + "_parse_datatype_json_string" + "\n").getBytes("utf-8")) + val schema = obj.asInstanceOf[StructType] + pickler.save(schema.json) + out.write(Opcodes.TUPLE1) + out.write(Opcodes.REDUCE) + } + } + + /** + * Pickler for InternalRow + */ + private class RowPickler extends IObjectPickler { + + private val cls = classOf[GenericInternalRowWithSchema] + + // register this to Pickler and Unpickler + def register(): Unit = { + Pickler.registerCustomPickler(this.getClass, this) + Pickler.registerCustomPickler(cls, this) + } + + def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write((module + "\n" + "_create_row_inbound_converter" + "\n").getBytes("utf-8")) + } else { + // it will be memorized by Pickler to save some bytes + pickler.save(this) + val row = obj.asInstanceOf[GenericInternalRowWithSchema] + // schema should always be same object for memoization + pickler.save(row.schema) + out.write(Opcodes.TUPLE1) + out.write(Opcodes.REDUCE) + + out.write(Opcodes.MARK) + var i = 0 + while (i < row.values.size) { + pickler.save(row.values(i)) + i += 1 + } + row.values.foreach(pickler.save) + out.write(Opcodes.TUPLE) + out.write(Opcodes.REDUCE) + } + } + } + + private[this] var registered = false + /** + * This should be called before trying to serialize any above classes un cluster mode, + * this should be put in the closure + */ + def registerPicklers(): Unit = { + synchronized { + if (!registered) { + SerDeUtil.initialize() + new StructTypePickler().register() + new RowPickler().register() + registered = true + } + } + } + + /** + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(rdd: RDD[Any]): RDD[Array[Byte]] = { + rdd.mapPartitions { iter => + registerPicklers() // let it called in executor + new SerDeUtil.AutoBatchedPickler(iter) + } + } +} + +/** + * :: DeveloperApi :: + * Evaluates a [[PythonUDF]], appending the result to the end of the input tuple. + */ +@DeveloperApi +case class EvaluatePython( + udf: PythonUDF, + child: LogicalPlan, + resultAttribute: AttributeReference) + extends logical.UnaryNode { + + def output: Seq[Attribute] = child.output :+ resultAttribute + + // References should not include the produced attribute. + override def references: AttributeSet = udf.references +} + +/** + * :: DeveloperApi :: + * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time. + * The input data is zipped with the result of the udf evaluation. + */ +@DeveloperApi +case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan) + extends SparkPlan { + + def children: Seq[SparkPlan] = child :: Nil + + protected override def doExecute(): RDD[InternalRow] = { + val childResults = child.execute().map(_.copy()) + + val parent = childResults.mapPartitions { iter => + EvaluatePython.registerPicklers() // register pickler for Row + val pickle = new Pickler + val currentRow = newMutableProjection(udf.children, child.output)() + val fields = udf.children.map(_.dataType) + val schema = new StructType(fields.map(t => new StructField("", t, true)).toArray) + iter.grouped(100).map { inputRows => + val toBePickled = inputRows.map { row => + EvaluatePython.toJava(currentRow(row), schema) + }.toArray + pickle.dumps(toBePickled) + } + } + + val pyRDD = new PythonRDD( + parent, + udf.command, + udf.envVars, + udf.pythonIncludes, + false, + udf.pythonExec, + udf.pythonVer, + udf.broadcastVars, + udf.accumulator + ).mapPartitions { iter => + val pickle = new Unpickler + iter.flatMap { pickedResult => + val unpickledBatch = pickle.loads(pickedResult) + unpickledBatch.asInstanceOf[java.util.ArrayList[Any]] + } + }.mapPartitions { iter => + val row = new GenericMutableRow(1) + iter.map { result => + row(0) = EvaluatePython.fromJava(result, udf.dataType) + row: InternalRow + } + } + + childResults.zip(pyRDD).mapPartitions { iter => + val joinedRow = new JoinedRow() + iter.map { + case (row, udfResult) => + joinedRow(row, udfResult) + } + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala new file mode 100644 index 0000000000000..3259b50acc765 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -0,0 +1,592 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ +import scala.util.Try + +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory +import org.apache.hadoop.hive.ql.exec._ +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} +import org.apache.hadoop.hive.ql.udf.generic._ +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper + +import org.apache.spark.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.types._ + + +private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry) + extends analysis.FunctionRegistry with HiveInspectors { + + def getFunctionInfo(name: String): FunctionInfo = FunctionRegistry.getFunctionInfo(name) + + override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + Try(underlying.lookupFunction(name, children)).getOrElse { + // We only look it up to see if it exists, but do not include it in the HiveUDF since it is + // not always serializable. + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse( + throw new AnalysisException(s"undefined function $name")) + + val functionClassName = functionInfo.getFunctionClass.getName + + if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), children) + } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveGenericUDF(new HiveFunctionWrapper(functionClassName), children) + } else if ( + classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveGenericUDAF(new HiveFunctionWrapper(functionClassName), children) + } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveUDAF(new HiveFunctionWrapper(functionClassName), children) + } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), children) + } else { + sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") + } + } + } + + override def registerFunction(name: String, builder: FunctionBuilder): Unit = + underlying.registerFunction(name, builder) +} + +private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression with HiveInspectors with CodegenFallback with Logging { + + type UDFType = UDF + + override def deterministic: Boolean = isUDFDeterministic + + override def nullable: Boolean = true + + @transient + lazy val function = funcWrapper.createFunction[UDFType]() + + @transient + protected lazy val method = + function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + + @transient + protected lazy val arguments = children.map(toInspector).toArray + + @transient + protected lazy val isUDFDeterministic = { + val udfType = function.getClass().getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() + } + + override def foldable: Boolean = isUDFDeterministic && children.forall(_.foldable) + + // Create parameter converters + @transient + protected lazy val conversionHelper = new ConversionHelper(method, arguments) + + @transient + lazy val dataType = javaClassToDataType(method.getReturnType) + + @transient + lazy val returnInspector = ObjectInspectorFactory.getReflectionObjectInspector( + method.getGenericReturnType(), ObjectInspectorOptions.JAVA) + + @transient + protected lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) + + // TODO: Finish input output types. + override def eval(input: InternalRow): Any = { + unwrap( + FunctionRegistry.invoke(method, function, conversionHelper + .convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*), + returnInspector) + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } +} + +// Adapter from Catalyst ExpressionResult to Hive DeferredObject +private[hive] class DeferredObjectAdapter(oi: ObjectInspector) + extends DeferredObject with HiveInspectors { + private var func: () => Any = _ + def set(func: () => Any): Unit = { + this.func = func + } + override def prepare(i: Int): Unit = {} + override def get(): AnyRef = wrap(func(), oi) +} + +private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression with HiveInspectors with CodegenFallback with Logging { + type UDFType = GenericUDF + + override def deterministic: Boolean = isUDFDeterministic + + override def nullable: Boolean = true + + @transient + lazy val function = funcWrapper.createFunction[UDFType]() + + @transient + protected lazy val argumentInspectors = children.map(toInspector) + + @transient + protected lazy val returnInspector = { + function.initializeAndFoldConstants(argumentInspectors.toArray) + } + + @transient + protected lazy val isUDFDeterministic = { + val udfType = function.getClass.getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() + } + + override def foldable: Boolean = + isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] + + @transient + protected lazy val deferedObjects = + argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] + + lazy val dataType: DataType = inspectorToDataType(returnInspector) + + override def eval(input: InternalRow): Any = { + returnInspector // Make sure initialized. + + var i = 0 + while (i < children.length) { + val idx = i + deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set( + () => { + children(idx).eval(input) + }) + i += 1 + } + unwrap(function.evaluate(deferedObjects), returnInspector) + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } +} + +/** + * Resolves [[UnresolvedWindowFunction]] to [[HiveWindowFunction]]. + */ +private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case p: LogicalPlan if !p.childrenResolved => p + + // We are resolving WindowExpressions at here. When we get here, we have already + // replaced those WindowSpecReferences. + case p: LogicalPlan => + p transformExpressions { + case WindowExpression( + UnresolvedWindowFunction(name, children), + windowSpec: WindowSpecDefinition) => + // First, let's find the window function info. + val windowFunctionInfo: WindowFunctionInfo = + Option(FunctionRegistry.getWindowFunctionInfo(name.toLowerCase)).getOrElse( + throw new AnalysisException(s"Couldn't find window function $name")) + + // Get the class of this function. + // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use + // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1. + val functionClass = windowFunctionInfo.getfInfo().getFunctionClass + val newChildren = + // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit + // input parameters and requires implicit parameters, which + // are expressions in Order By clause. + if (classOf[GenericUDAFRank].isAssignableFrom(functionClass)) { + if (children.nonEmpty) { + throw new AnalysisException(s"$name does not take input parameters.") + } + windowSpec.orderSpec.map(_.child) + } else { + children + } + + // If the class is UDAF, we need to use UDAFBridge. + val isUDAFBridgeRequired = + if (classOf[UDAF].isAssignableFrom(functionClass)) { + true + } else { + false + } + + // Create the HiveWindowFunction. For the meaning of isPivotResult, see the doc of + // HiveWindowFunction. + val windowFunction = + HiveWindowFunction( + new HiveFunctionWrapper(functionClass.getName), + windowFunctionInfo.isPivotResult, + isUDAFBridgeRequired, + newChildren) + + // Second, check if the specified window function can accept window definition. + windowSpec.frameSpecification match { + case frame: SpecifiedWindowFrame if !windowFunctionInfo.isSupportsWindow => + // This Hive window function does not support user-speficied window frame. + throw new AnalysisException( + s"Window function $name does not take a frame specification.") + case frame: SpecifiedWindowFrame if windowFunctionInfo.isSupportsWindow && + windowFunctionInfo.isPivotResult => + // These two should not be true at the same time when a window frame is defined. + // If so, throw an exception. + throw new AnalysisException(s"Could not handle Hive window function $name because " + + s"it supports both a user specified window frame and pivot result.") + case _ => // OK + } + // Resolve those UnspecifiedWindowFrame because the physical Window operator still needs + // a window frame specification to work. + val newWindowSpec = windowSpec.frameSpecification match { + case UnspecifiedFrame => + val newWindowFrame = + SpecifiedWindowFrame.defaultWindowFrame( + windowSpec.orderSpec.nonEmpty, + windowFunctionInfo.isSupportsWindow) + WindowSpecDefinition(windowSpec.partitionSpec, windowSpec.orderSpec, newWindowFrame) + case _ => windowSpec + } + + // Finally, we create a WindowExpression with the resolved window function and + // specified window spec. + WindowExpression(windowFunction, newWindowSpec) + } + } +} + +/** + * A [[WindowFunction]] implementation wrapping Hive's window function. + * @param funcWrapper The wrapper for the Hive Window Function. + * @param pivotResult If it is true, the Hive function will return a list of values representing + * the values of the added columns. Otherwise, a single value is returned for + * current row. + * @param isUDAFBridgeRequired If it is true, the function returned by functionWrapper's + * createFunction is UDAF, we need to use GenericUDAFBridge to wrap + * it as a GenericUDAFResolver2. + * @param children Input parameters. + */ +private[hive] case class HiveWindowFunction( + funcWrapper: HiveFunctionWrapper, + pivotResult: Boolean, + isUDAFBridgeRequired: Boolean, + children: Seq[Expression]) extends WindowFunction + with HiveInspectors with Unevaluable { + + // Hive window functions are based on GenericUDAFResolver2. + type UDFType = GenericUDAFResolver2 + + @transient + protected lazy val resolver: GenericUDAFResolver2 = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) + } else { + funcWrapper.createFunction[GenericUDAFResolver2]() + } + + @transient + protected lazy val inputInspectors = children.map(toInspector).toArray + + // The GenericUDAFEvaluator used to evaluate the window function. + @transient + protected lazy val evaluator: GenericUDAFEvaluator = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + // The object inspector of values returned from the Hive window function. + @transient + protected lazy val returnInspector = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + override def dataType: DataType = + if (!pivotResult) { + inspectorToDataType(returnInspector) + } else { + // If pivotResult is true, we should take the element type out as the data type of this + // function. + inspectorToDataType(returnInspector) match { + case ArrayType(dt, _) => dt + case _ => + sys.error( + s"error resolve the data type of window function ${funcWrapper.functionClassName}") + } + } + + override def nullable: Boolean = true + + @transient + lazy val inputProjection = new InterpretedProjection(children) + + @transient + private var hiveEvaluatorBuffer: AggregationBuffer = _ + // Output buffer. + private var outputBuffer: Any = _ + + override def init(): Unit = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + // Reset the hiveEvaluatorBuffer and outputPosition + override def reset(): Unit = { + // We create a new aggregation buffer to workaround the bug in GenericUDAFRowNumber. + // Basically, GenericUDAFRowNumberEvaluator.reset calls RowNumberBuffer.init. + // However, RowNumberBuffer.init does not really reset this buffer. + hiveEvaluatorBuffer = evaluator.getNewAggregationBuffer + evaluator.reset(hiveEvaluatorBuffer) + } + + override def prepareInputParameters(input: InternalRow): AnyRef = { + wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length)) + } + // Add input parameters for a single row. + override def update(input: AnyRef): Unit = { + evaluator.iterate(hiveEvaluatorBuffer, input.asInstanceOf[Array[AnyRef]]) + } + + override def batchUpdate(inputs: Array[AnyRef]): Unit = { + var i = 0 + while (i < inputs.length) { + evaluator.iterate(hiveEvaluatorBuffer, inputs(i).asInstanceOf[Array[AnyRef]]) + i += 1 + } + } + + override def evaluate(): Unit = { + outputBuffer = unwrap(evaluator.evaluate(hiveEvaluatorBuffer), returnInspector) + } + + override def get(index: Int): Any = { + if (!pivotResult) { + // if pivotResult is false, we will get a single value for all rows in the frame. + outputBuffer + } else { + // if pivotResult is true, we will get a Seq having the same size with the size + // of the window frame. At here, we will return the result at the position of + // index in the output buffer. + outputBuffer.asInstanceOf[Seq[Any]].get(index) + } + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def newInstance(): WindowFunction = + new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children) +} + +private[hive] case class HiveGenericUDAF( + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) extends AggregateExpression1 + with HiveInspectors { + + type UDFType = AbstractGenericUDAFResolver + + @transient + protected lazy val resolver: AbstractGenericUDAFResolver = funcWrapper.createFunction() + + @transient + protected lazy val objectInspector = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors.toArray, false, false) + resolver.getEvaluator(parameterInfo) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + @transient + protected lazy val inspectors = children.map(toInspector) + + def dataType: DataType = inspectorToDataType(objectInspector) + + def nullable: Boolean = true + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + def newInstance(): HiveUDAFFunction = new HiveUDAFFunction(funcWrapper, children, this) +} + +/** It is used as a wrapper for the hive functions which uses UDAF interface */ +private[hive] case class HiveUDAF( + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) extends AggregateExpression1 + with HiveInspectors { + + type UDFType = UDAF + + @transient + protected lazy val resolver: AbstractGenericUDAFResolver = + new GenericUDAFBridge(funcWrapper.createFunction()) + + @transient + protected lazy val objectInspector = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors.toArray, false, false) + resolver.getEvaluator(parameterInfo) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + @transient + protected lazy val inspectors = children.map(toInspector) + + def dataType: DataType = inspectorToDataType(objectInspector) + + def nullable: Boolean = true + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + def newInstance(): HiveUDAFFunction = new HiveUDAFFunction(funcWrapper, children, this, true) +} + +/** + * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a + * [[Generator]]. Note that the semantics of Generators do not allow + * Generators to maintain state in between input rows. Thus UDTFs that rely on partitioning + * dependent operations like calls to `close()` before producing output will not operate the same as + * in Hive. However, in practice this should not affect compatibility for most sane UDTFs + * (e.g. explode or GenericUDTFParseUrlTuple). + * + * Operators that require maintaining state in between input rows should instead be implemented as + * user defined aggregations, which have clean semantics even in a partitioned execution. + */ +private[hive] case class HiveGenericUDTF( + funcWrapper: HiveFunctionWrapper, + children: Seq[Expression]) + extends Generator with HiveInspectors with CodegenFallback { + + @transient + protected lazy val function: GenericUDTF = { + val fun: GenericUDTF = funcWrapper.createFunction() + fun.setCollector(collector) + fun + } + + @transient + protected lazy val inputInspectors = children.map(toInspector) + + @transient + protected lazy val outputInspector = function.initialize(inputInspectors.toArray) + + @transient + protected lazy val udtInput = new Array[AnyRef](children.length) + + @transient + protected lazy val collector = new UDTFCollector + + lazy val elementTypes = outputInspector.getAllStructFieldRefs.map { + field => (inspectorToDataType(field.getFieldObjectInspector), true) + } + + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { + outputInspector // Make sure initialized. + + val inputProjection = new InterpretedProjection(children) + + function.process(wrap(inputProjection(input), inputInspectors, udtInput)) + collector.collectRows() + } + + protected class UDTFCollector extends Collector { + var collected = new ArrayBuffer[InternalRow] + + override def collect(input: java.lang.Object) { + // We need to clone the input here because implementations of + // GenericUDTF reuse the same object. Luckily they are always an array, so + // it is easy to clone. + collected += unwrap(input, outputInspector).asInstanceOf[InternalRow] + } + + def collectRows(): Seq[InternalRow] = { + val toCollect = collected + collected = new ArrayBuffer[InternalRow] + toCollect + } + } + + override def terminate(): TraversableOnce[InternalRow] = { + outputInspector // Make sure initialized. + function.close() + collector.collectRows() + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } +} + +private[hive] case class HiveUDAFFunction( + funcWrapper: HiveFunctionWrapper, + exprs: Seq[Expression], + base: AggregateExpression1, + isUDAFBridgeRequired: Boolean = false) + extends AggregateFunction1 + with HiveInspectors { + + def this() = this(null, null, null) + + private val resolver = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) + } else { + funcWrapper.createFunction[AbstractGenericUDAFResolver]() + } + + private val inspectors = exprs.map(toInspector).toArray + + private val function = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) + + private val buffer = + function.getNewAggregationBuffer + + override def eval(input: InternalRow): Any = unwrap(function.evaluate(buffer), returnInspector) + + @transient + val inputProjection = new InterpretedProjection(exprs) + + @transient + protected lazy val cached = new Array[AnyRef](exprs.length) + + def update(input: InternalRow): Unit = { + val inputs = inputProjection(input) + function.iterate(buffer, wrap(inputs, inspectors, cached)) + } +} + From ee8d52e8f66796b213a83fa19e85e9a080143e66 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 24 Jul 2015 11:02:15 -0500 Subject: [PATCH 20/45] Updated test suite to properly create task sets and force stage failures --- .../spark/scheduler/DAGSchedulerSuite.scala | 106 +++++++++--------- 1 file changed, 54 insertions(+), 52 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 053ec4d6f583b..fa802ff6222d4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -495,8 +495,8 @@ class DAGSchedulerSuite submit(reduceRdd, Array(0, 1)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) for (x <- 1 to Stage.MAX_STAGE_FAILURES) { // the 2nd ResultTask failed @@ -504,6 +504,9 @@ class DAGSchedulerSuite (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + println(s"iteration $x: ${taskSets.size} taskSets") + taskSets.zipWithIndex.foreach{ case (ts, idx) => println(s"$idx: $ts")} + scheduler.resubmitFailedStages() if (x < Stage.MAX_STAGE_FAILURES) { assert(scheduler.runningStages.nonEmpty) @@ -519,7 +522,7 @@ class DAGSchedulerSuite } - test("Multiple consecutive Fetch failures in a stage should trigger an abort.") { + test("Multiple consecutive Fetch failures in a stage should abort on attempt 4, not on 1.") { // 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. @@ -534,37 +537,40 @@ class DAGSchedulerSuite sc.listenerBus.addListener(new EndListener()) - val shuffleMapRdd = new MyRDD(sc, 8, Nil) + val shuffleMapRdd = new MyRDD(sc, 6, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = new MyRDD(sc, 8, List(shuffleDep)) - submit(reduceRdd, Array(0, 1, 2, 3, 4, 5, 6, 7)) + val reduceRdd = new MyRDD(sc, 6, List(shuffleDep)) + submit(reduceRdd, Array(0, 1, 2, 3, 4, 5)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) - complete(taskSets(1), Seq( - (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored1"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored2"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored3"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored4"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored5"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored6"), null))) + for (x <- 1 to Stage.MAX_STAGE_FAILURES) { + complete(taskSets(1), Seq( + (Success, 42), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored1"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored2"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored3"), null), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored4"), null))) - scheduler.resubmitFailedStages() - assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(1000) - assert(ended) - assert(jobResult.isInstanceOf[JobFailed]) + if (x < 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]) + } + } } test("Multiple consecutive task failures (not FetchFailures) in a stage should not " + @@ -582,36 +588,32 @@ class DAGSchedulerSuite } sc.listenerBus.addListener(new EndListener()) - - val shuffleMapRdd = new MyRDD(sc, 8, Nil) + val shuffleMapRdd = new MyRDD(sc, 6, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = new MyRDD(sc, 8, List(shuffleDep)) - submit(reduceRdd, Array(0, 1, 2, 3, 4, 5, 6, 7)) + val reduceRdd = new MyRDD(sc, 6, List(shuffleDep)) + submit(reduceRdd, Array(0, 1, 2, 3, 4, 5)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) - complete(taskSets(1), Seq( - (Success, 42), - (ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failB", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failC", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failD", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failE", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failF", null, "This is a stack.", None), null), - (Success, 43))) + for (x <- 1 to Stage.MAX_STAGE_FAILURES+1) { + complete(taskSets(1), Seq( + (Success, 42), + (ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failB", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failE", null, "This is a stack.", None), null), + (ExceptionFailure("fakeExcept", "failF", null, "This is a stack.", None), null), + (Success, 43))) - scheduler.resubmitFailedStages() - assert(scheduler.runningStages.nonEmpty) - assert(!ended) + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + } } test("trivial shuffle with multiple fetch failures") { From 4da3d5d489c14cbed812c3ca8a2e0cd46f7e4450 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 24 Jul 2015 11:03:32 -0500 Subject: [PATCH 21/45] got rid of println --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index fa802ff6222d4..cf8dee733fdc8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -503,10 +503,7 @@ class DAGSchedulerSuite complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) - - println(s"iteration $x: ${taskSets.size} taskSets") - taskSets.zipWithIndex.foreach{ case (ts, idx) => println(s"$idx: $ts")} - + scheduler.resubmitFailedStages() if (x < Stage.MAX_STAGE_FAILURES) { assert(scheduler.runningStages.nonEmpty) From 5e13342b6d5a4a68324997c6fd190616ce442bd1 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 27 Jul 2015 13:49:25 -0700 Subject: [PATCH 22/45] Updated tests for stage failures. --- .../spark/scheduler/DAGSchedulerSuite.scala | 204 +++++++++++++----- 1 file changed, 145 insertions(+), 59 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index cf8dee733fdc8..a45528f6aa307 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -473,7 +473,22 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } - test("Multiple consecutive stage failures should lead to stage being aborted.") { + // Helper function to validate state and print output when creating tests for task failures + def checkStageIdAndPrint(stageId: Int, attempt: Int, stageAttempt: TaskSet) { + println(s"$attempt($attempt): taskSets = $taskSets : ${ + taskSets.map{_.tasks.mkString(",")}.mkString(";")}") + + assert(stageAttempt.stageId === stageId) + assert(stageAttempt.stageAttemptId == attempt) + println(s"tasks for $stageAttempt : ${stageAttempt.tasks.mkString(",")}") + } + + /** + * 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. @@ -494,18 +509,39 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + for (attempt <- 0 to Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last - for (x <- 1 to Stage.MAX_STAGE_FAILURES) { - // the 2nd ResultTask failed - complete(taskSets(1), Seq( - (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) - + // Confirm that this is the first attempt for stage 0 + checkStageIdAndPrint(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 + checkStageIdAndPrint(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 (x < Stage.MAX_STAGE_FAILURES) { + + if (attempt < Stage.MAX_STAGE_FAILURES) { assert(scheduler.runningStages.nonEmpty) assert(!ended) } else { @@ -518,8 +554,13 @@ class DAGSchedulerSuite } } - - test("Multiple consecutive Fetch failures in a stage should abort on attempt 4, not on 1.") { + /** + * 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") { // 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. @@ -534,42 +575,66 @@ class DAGSchedulerSuite sc.listenerBus.addListener(new EndListener()) - val shuffleMapRdd = new MyRDD(sc, 6, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = new MyRDD(sc, 6, List(shuffleDep)) - submit(reduceRdd, Array(0, 1, 2, 3, 4, 5)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + for (attempt <- 0 to Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last - for (x <- 1 to Stage.MAX_STAGE_FAILURES) { - complete(taskSets(1), Seq( - (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored1"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored2"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored3"), null), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored4"), null))) + // Confirm that this is the first attempt for stage 0 + checkStageIdAndPrint(0, attempt, stage0Attempt) - if (x < Stage.MAX_STAGE_FAILURES) { - assert(scheduler.runningStages.nonEmpty) - assert(!ended) + 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 + checkStageIdAndPrint(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 { - // Stage has been aborted and removed from running stages - assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(1000) - assert(ended) - assert(jobResult.isInstanceOf[JobFailed]) + 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 + ) } + + // 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() + + // In this test case, we're confirming that we never actually fail out (so we should have a + // stage that has been resubmitted) + assert(scheduler.runningStages.nonEmpty) + assert(!ended) } } + /** + * In this test we simulate a job failure where the first stage completes successfully and + * the second stage fails due to an exception. Multiple successive failures are still allowed in + * this circumstance since they're not due to a fetch failure. + */ test("Multiple consecutive task failures (not FetchFailures) in a stage should not " + "trigger an abort.") { // Create a new Listener to confirm that the listenerBus sees the JobEnd message @@ -585,34 +650,55 @@ class DAGSchedulerSuite } sc.listenerBus.addListener(new EndListener()) - val shuffleMapRdd = new MyRDD(sc, 6, Nil) + + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = new MyRDD(sc, 6, List(shuffleDep)) - submit(reduceRdd, Array(0, 1, 2, 3, 4, 5)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + for (attempt <- 0 to Stage.MAX_STAGE_FAILURES) { + // Complete all the tasks for the current attempt of stage 0 successfully + val stage0Attempt = taskSets.last - for (x <- 1 to Stage.MAX_STAGE_FAILURES+1) { - complete(taskSets(1), Seq( - (Success, 42), - (ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failB", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failE", null, "This is a stack.", None), null), - (ExceptionFailure("fakeExcept", "failF", null, "This is a stack.", None), null), - (Success, 43))) + // Confirm that this is the first attempt for stage 0 + checkStageIdAndPrint(0, attempt, stage0Attempt) - assert(scheduler.runningStages.nonEmpty) - assert(!ended) + // 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, and we'll complete the other tasks + // normally. + val stage1Attempt = taskSets.last + checkStageIdAndPrint(1, attempt, stage1Attempt) + + val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, 42)} + + // Run Stage 1, this time with an exception failure + complete(stage1Attempt, + Seq((ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), 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() + + // In this test case, we're confirming that we never actually fail out (so we should have a + // stage that has been resubmitted) + assert(scheduler.runningStages.nonEmpty) + assert(!ended) } } + + test("trivial shuffle with multiple fetch failures") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From e101ed752adc6f1a4a33d653cb24f8a5de88d4a8 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 28 Jul 2015 09:22:50 -0700 Subject: [PATCH 23/45] Updated test for fetch failures. Added validation of successfully generated output --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../org/apache/spark/scheduler/Stage.scala | 28 +++-- .../spark/scheduler/DAGSchedulerSuite.scala | 107 ++++-------------- 3 files changed, 41 insertions(+), 98 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0bd3e181e619d..3a6d240b73a2e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1066,7 +1066,7 @@ class DAGScheduler( if (disallowStageRetryForTest) { abortStage(failedStage, "Fetch failure will not retry stage due to testing config") - } else if (failedStage.failAndShouldAbort()) { + } else if (failedStage.failAndShouldAbort(task)) { abortStage(failedStage, s"Stage ${failedStage.name} " + s"has failed the maximum allowable number of times: ${Stage.MAX_STAGE_FAILURES}. " + s"Most recent failure reason: ${failureMessage}") @@ -1197,7 +1197,7 @@ class DAGScheduler( // Clear failure count for this stage, now that it's succeeded. // We only limit consecutive failures of stage attempts,so that if a stage is // re-used many times in a long-running job, unrelated failures that are spaced out in time - // don't eventually cause the stage to be aborted." + // don't eventually cause the stage to be aborted. stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index f90e9f018a76b..3538de9e10d89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -46,7 +46,7 @@ import scala.collection.mutable.HashSet * be updated for each attempt. * */ -private[spark] abstract class Stage( +private[scheduler] abstract class Stage( val id: Int, val rdd: RDD[_], val numTasks: Int, @@ -78,29 +78,33 @@ private[spark] abstract class Stage( /** * Spark is resilient to executors dying by retrying stages on FetchFailures. Here, we keep track - * of the number of stage failures to prevent endless stage retries. + * of unique stage failures (per stage attempt) triggered by fetch failures to prevent endless + * stage retries. Specifically, per stage we wish to only record a failure when the following + * holds: + * + * A) A fetch failure was observed + * B) A failure has not yet been registered for this stage attempt. There may be multiple + * concurrent failures for a sinlge stage since we may have multiple tasks executing at the same + * time, one or many of which may fail. Also, even though there may only be one non-zombie stage + * attemp, zombie stages may still have running tasks. */ - private var failedStageCount = 0 + private val attemptsFailedFromFetch = new HashSet[Int] private[scheduler] def clearFailures() : Unit = { - failedStageCount = 0 + attemptsFailedFromFetch.clear() } /** * Check whether we should abort the failedStage due to multiple failures. - * This method updates the running count of failures for a particular stage and returns + * This method updates the running set of failures for a particular stage and returns * true if the number of failures exceeds the allowable number of failures. */ - private[scheduler] def failAndShouldAbort(): Boolean = { + private[scheduler] def failAndShouldAbort(task: Task[_]): Boolean = { // We increment the failure count on the first attempt for a particular Stage - if (_latestInfo.attemptId == 0) - { - failedStageCount += 1 - } - + attemptsFailedFromFetch.add(task.stageAttemptId) // Check for multiple FetchFailures in a Stage and for the stage failing repeatedly following // resubmissions. - failedStageCount >= Stage.MAX_STAGE_FAILURES + attemptsFailedFromFetch.size >= Stage.MAX_STAGE_FAILURES } /** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index a45528f6aa307..262020aa10558 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -26,11 +26,11 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite -import org.apache.spark.executor.TaskMetrics class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { @@ -209,7 +209,7 @@ class DAGSchedulerSuite /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { - assert(taskSet.tasks.size >= results.size) + assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { runEvent(CompletionEvent( @@ -473,14 +473,10 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } - // Helper function to validate state and print output when creating tests for task failures - def checkStageIdAndPrint(stageId: Int, attempt: Int, stageAttempt: TaskSet) { - println(s"$attempt($attempt): taskSets = $taskSets : ${ - taskSets.map{_.tasks.mkString(",")}.mkString(";")}") - + // 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) - println(s"tasks for $stageAttempt : ${stageAttempt.tasks.mkString(",")}") + assert(stageAttempt.stageAttemptId == attempt-1) } /** @@ -509,12 +505,12 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - for (attempt <- 0 to Stage.MAX_STAGE_FAILURES) { + 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 - checkStageIdAndPrint(0, attempt, stage0Attempt) + checkStageId(0, attempt, stage0Attempt) // Make each task in stage 0 success val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => @@ -527,7 +523,7 @@ class DAGSchedulerSuite // 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 - checkStageIdAndPrint(1, attempt, stage1Attempt) + checkStageId(1, attempt, stage1Attempt) val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, 42)} @@ -581,12 +577,14 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - for (attempt <- 0 to Stage.MAX_STAGE_FAILURES) { + // 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 - checkStageIdAndPrint(0, attempt, stage0Attempt) + checkStageId(0, attempt, stage0Attempt) if (attempt < Stage.MAX_STAGE_FAILURES/2) { // Make each task in stage 0 success @@ -600,7 +598,7 @@ class DAGSchedulerSuite // 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 - checkStageIdAndPrint(1, attempt, stage1Attempt) + checkStageId(1, attempt, stage1Attempt) val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, 42)} @@ -622,83 +620,24 @@ class DAGSchedulerSuite // 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() - - // In this test case, we're confirming that we never actually fail out (so we should have a - // stage that has been resubmitted) - assert(scheduler.runningStages.nonEmpty) - assert(!ended) } - } - - /** - * In this test we simulate a job failure where the first stage completes successfully and - * the second stage fails due to an exception. Multiple successive failures are still allowed in - * this circumstance since they're not due to a fetch failure. - */ - test("Multiple consecutive task failures (not FetchFailures) in a stage should not " + - "trigger an abort.") { - // 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 <- 0 to Stage.MAX_STAGE_FAILURES) { - // Complete all the tasks for the current attempt of stage 0 successfully - val stage0Attempt = taskSets.last + val stage0Attempt = taskSets.last + val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, 2)) + }.toSeq - // Confirm that this is the first attempt for stage 0 - checkStageIdAndPrint(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 + // Complete first task + complete(taskSets.last, completions) - // Run stage 0 - complete(stage0Attempt, completions) + // Complete second task + complete(taskSets.last, Seq((Success, 42))) - // Now we should have a new taskSet, for a new attempt of stage 1. - // We will have one fetch failure for this task set, and we'll complete the other tasks - // normally. - val stage1Attempt = taskSets.last - checkStageIdAndPrint(1, attempt, stage1Attempt) + // The first success is from the success we append in stage 1, the second is the one we add here + assert(results === Map(1 -> 42, 0 -> 42)) - val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, 42)} - - // Run Stage 1, this time with an exception failure - complete(stage1Attempt, - Seq((ExceptionFailure("fakeExcept", "failA", null, "This is a stack.", None), 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() - - // In this test case, we're confirming that we never actually fail out (so we should have a - // stage that has been resubmitted) - assert(scheduler.runningStages.nonEmpty) - assert(!ended) - } } - - test("trivial shuffle with multiple fetch failures") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From daad2e49130272f18eeede6f5c983cf4b1598bd0 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 28 Jul 2015 09:56:36 -0700 Subject: [PATCH 24/45] Added a test for multiple fetch failures inside a single stage --- .../spark/scheduler/DAGSchedulerSuite.scala | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 262020aa10558..28ff463257e46 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -638,6 +638,74 @@ class DAGSchedulerSuite } + /** + * 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.") { + // 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 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 stage0Attempt = taskSets.last + + // Make each task in stage 0 success, then fail all of stage 1 + val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, parts)) + }.toSeq + + complete(stage0Attempt, completions) + + val stage1Attempt = taskSets.last + val failures = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, idx, "ignored"), null) + }.toSeq + + // Run Stage 1, this time with all fetchs failing + complete(stage1Attempt, failures) + + // Resubmit and confirm that now all is well + scheduler.resubmitFailedStages() + + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + + // Confirm job finished succesfully + val stage0Attempt2 = taskSets.last + val completions2 = stage0Attempt2.tasks.zipWithIndex.map{ case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, parts)) + }.toSeq + + complete(taskSets.last, completions2) + + val stage1Attempt2 = taskSets.last + val completions3 = stage1Attempt2.tasks.zipWithIndex.map{ case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, parts)) + }.toSeq + + complete(taskSets.last, completions3) + sc.listenerBus.waitUntilEmpty(1000) + assert(!jobResult.isInstanceOf[JobFailed]) + } + test("trivial shuffle with multiple fetch failures") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 0c054d36c7a0ee6693553d805e3a0f309f6f44db Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 28 Jul 2015 11:01:20 -0700 Subject: [PATCH 25/45] Added a test for multiple fetch failures inside a single stage --- .../spark/scheduler/DAGSchedulerSuite.scala | 111 +++++++++++++++--- 1 file changed, 96 insertions(+), 15 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 28ff463257e46..412d3ecd18bbe 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -557,20 +557,6 @@ class DAGSchedulerSuite * aborted. */ test("Failures in different stages should not trigger an overall abort") { - // 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 @@ -635,7 +621,6 @@ class DAGSchedulerSuite // The first success is from the success we append in stage 1, the second is the one we add here assert(results === Map(1 -> 42, 0 -> 42)) - } /** @@ -704,6 +689,102 @@ class DAGSchedulerSuite complete(taskSets.last, completions3) sc.listenerBus.waitUntilEmpty(1000) assert(!jobResult.isInstanceOf[JobFailed]) + assert(ended === true) + } + + /** + * In this test we demonstrate that only consecutive failures trigger a stage abort. In short, a + * job may fail multiple times, succeed once in a previously failed stage, then fail again, then + * complete both stages successfully without aborting. + */ + test("Abort should only trigger after consecutive stage failures") { + // 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 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)) + + // First, fail stage 0 multiple times after suceeding stage 2 and 1 + for (attempt <- 1 to Stage.MAX_STAGE_FAILURES-1) { + println(s"$attempt: taskSets = $taskSets : ${ + taskSets.map{_.tasks.mkString(",")}.mkString(";")}") + + // complete stage 2 + complete(taskSets.last, Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + + // Pretend stage one fails + complete(taskSets.last, Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null))) + + scheduler.resubmitFailedStages() + // Now complete stage 0 + complete(taskSets.last, Seq((Success, 42))) + + // Confirm we have not yet aborted + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + } + + // Now succeed stage 1 and fail stage 0 + + // complete stage 2 + complete(taskSets.last, Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + // complete stage 1 + complete(taskSets.last, Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + // pretend stage 0 failed because hostA went down + complete(taskSets.last, Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + + scheduler.resubmitFailedStages() + + // Now again, fail stage 1 (up to MAX_FAILURES) but confirm that this doesn't trigger an abort + // since we succeeded in between + complete(taskSets.last, Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + + // Pretend stage one fails + complete(taskSets.last, Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null))) + + // Drop the uncompleted stage 0 + taskSets.dropRight(1) + + scheduler.resubmitFailedStages() + + // Confirm we have not yet aborted + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + + // Next, succeed all and confirm output + complete(taskSets.last, Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + complete(taskSets.last, Seq((Success, makeMapStatus("hostD", 1)))) + complete(taskSets.last, Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty() } test("trivial shuffle with multiple fetch failures") { From f23c31b8caa4184befd3905deb46d7ac6a7de6f8 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 28 Jul 2015 11:07:02 -0700 Subject: [PATCH 26/45] Added test to ensure that stage failure only triggers with successive fetch failures. NOTE: This test does not presently work --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 412d3ecd18bbe..54c64687108f7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -721,9 +721,6 @@ class DAGSchedulerSuite // First, fail stage 0 multiple times after suceeding stage 2 and 1 for (attempt <- 1 to Stage.MAX_STAGE_FAILURES-1) { - println(s"$attempt: taskSets = $taskSets : ${ - taskSets.map{_.tasks.mkString(",")}.mkString(";")}") - // complete stage 2 complete(taskSets.last, Seq( (Success, makeMapStatus("hostA", 2)), @@ -733,10 +730,12 @@ class DAGSchedulerSuite complete(taskSets.last, Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null))) + complete(taskSets.last, Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + scheduler.resubmitFailedStages() // Now complete stage 0 - complete(taskSets.last, Seq((Success, 42))) - + // Confirm we have not yet aborted assert(scheduler.runningStages.nonEmpty) assert(!ended) From 9978575febde94d08213a89592cf7c3b0d2cc45e Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 28 Jul 2015 16:57:31 -0700 Subject: [PATCH 27/45] Test updates. Some tests still failing, unsure why. --- .../org/apache/spark/scheduler/Stage.scala | 5 +- .../spark/scheduler/DAGSchedulerSuite.scala | 157 +++++++++--------- 2 files changed, 81 insertions(+), 81 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 3538de9e10d89..0bc90ed786125 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -96,14 +96,11 @@ private[scheduler] abstract class Stage( /** * Check whether we should abort the failedStage due to multiple failures. - * This method updates the running set of failures for a particular stage and returns + * This method updates the running set of failed stage attempts and returns * true if the number of failures exceeds the allowable number of failures. */ private[scheduler] def failAndShouldAbort(task: Task[_]): Boolean = { - // We increment the failure count on the first attempt for a particular Stage attemptsFailedFromFetch.add(task.stageAttemptId) - // Check for multiple FetchFailures in a Stage and for the stage failing repeatedly following - // resubmissions. attemptsFailedFromFetch.size >= Stage.MAX_STAGE_FAILURES } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 54c64687108f7..242bd4e695cd0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -209,7 +209,7 @@ class DAGSchedulerSuite /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { - assert(taskSet.tasks.size >= results.size) + assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { runEvent(CompletionEvent( @@ -476,7 +476,13 @@ class DAGSchedulerSuite // 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) + assert(stageAttempt.stageAttemptId == attempt) + } + + def makeCompletions(stageAttempt: TaskSet): Seq[(Success.type, MapStatus)] = { + stageAttempt.tasks.zipWithIndex.map { case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, stageAttempt.tasks.size)) + }.toSeq } /** @@ -505,17 +511,15 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - for (attempt <- 1 to Stage.MAX_STAGE_FAILURES) { + for (attempt <- 0 until 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 + // Confirm that this is the next 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 + val completions = makeCompletions(stage0Attempt) // Run stage 0 complete(stage0Attempt, completions) @@ -537,7 +541,7 @@ class DAGSchedulerSuite // map output, for the next iteration through the loop scheduler.resubmitFailedStages() - if (attempt < Stage.MAX_STAGE_FAILURES) { + if (attempt < Stage.MAX_STAGE_FAILURES-1) { assert(scheduler.runningStages.nonEmpty) assert(!ended) } else { @@ -565,18 +569,16 @@ class DAGSchedulerSuite // 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) { + for (attempt <- 0 until 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 + // Confirm that this is the next 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 + val completions = makeCompletions(stage0Attempt) // Run stage 0 complete(stage0Attempt, completions) @@ -626,7 +628,7 @@ class DAGSchedulerSuite /** * 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 . + * their own, but only when the stage fails enough times. */ test("Multiple task failures in same stage should not abort the stage.") { // Create a new Listener to confirm that the listenerBus sees the JobEnd message @@ -643,7 +645,7 @@ class DAGSchedulerSuite sc.listenerBus.addListener(new EndListener()) - val parts = 8; + val parts = 8 val shuffleMapRdd = new MyRDD(sc, parts, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId @@ -653,14 +655,11 @@ class DAGSchedulerSuite val stage0Attempt = taskSets.last // Make each task in stage 0 success, then fail all of stage 1 - val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, parts)) - }.toSeq - + val completions = makeCompletions(stage0Attempt) complete(stage0Attempt, completions) val stage1Attempt = taskSets.last - val failures = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + val failures = stage1Attempt.tasks.zipWithIndex.map{ case (task, idx) => (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, idx, "ignored"), null) }.toSeq @@ -674,28 +673,20 @@ class DAGSchedulerSuite assert(!ended) // Confirm job finished succesfully - val stage0Attempt2 = taskSets.last - val completions2 = stage0Attempt2.tasks.zipWithIndex.map{ case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, parts)) - }.toSeq - - complete(taskSets.last, completions2) - val stage1Attempt2 = taskSets.last - val completions3 = stage1Attempt2.tasks.zipWithIndex.map{ case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, parts)) - }.toSeq + val completions_1_2 = stage1Attempt2.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq - complete(taskSets.last, completions3) + complete(stage1Attempt2, completions_1_2) sc.listenerBus.waitUntilEmpty(1000) - assert(!jobResult.isInstanceOf[JobFailed]) assert(ended === true) + assert(results === Map(1 -> 42)) } /** - * In this test we demonstrate that only consecutive failures trigger a stage abort. In short, a - * job may fail multiple times, succeed once in a previously failed stage, then fail again, then - * complete both stages successfully without aborting. + * 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") { // Create a new Listener to confirm that the listenerBus sees the JobEnd message @@ -719,39 +710,46 @@ class DAGSchedulerSuite val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) - // First, fail stage 0 multiple times after suceeding stage 2 and 1 - for (attempt <- 1 to Stage.MAX_STAGE_FAILURES-1) { - // complete stage 2 - complete(taskSets.last, Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + for (attempt <- 0 until Stage.MAX_STAGE_FAILURES-1) { + println(s"attempt = $attempt") + println(taskSets.mkString(",")) - // Pretend stage one fails - complete(taskSets.last, Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null))) + // Make each task in stage 0 success + val stage0Attempt = taskSets.last - complete(taskSets.last, Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + val completions = makeCompletions(stage0Attempt); + + // 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 + 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() - // Now complete stage 0 // Confirm we have not yet aborted assert(scheduler.runningStages.nonEmpty) assert(!ended) } - // Now succeed stage 1 and fail stage 0 + // Now succeed stage 1 and fail stage 2 + val stage1attempt4 = taskSets.last + val completions_1_4 = makeCompletions(stage1attempt4) - // complete stage 2 - complete(taskSets.last, Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) - // complete stage 1 - complete(taskSets.last, Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) - // pretend stage 0 failed because hostA went down + // Succeed 1 + complete(stage1attempt4, completions_1_4) + + // Fail stage 2 complete(taskSets.last, Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) @@ -759,31 +757,36 @@ class DAGSchedulerSuite // Now again, fail stage 1 (up to MAX_FAILURES) but confirm that this doesn't trigger an abort // since we succeeded in between - complete(taskSets.last, Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) - - // Pretend stage one fails - complete(taskSets.last, Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null))) - - // Drop the uncompleted stage 0 - taskSets.dropRight(1) - + val stage1attempt5 = taskSets.last + val stage1Successes = + stage1attempt5.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} + + // Run Stage 1, this time with a task failure + complete(stage1attempt5, + Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null) + ) ++ stage1Successes + ) + + Thread.sleep(500) scheduler.resubmitFailedStages() - + Thread.sleep(500) // Confirm we have not yet aborted assert(scheduler.runningStages.nonEmpty) assert(!ended) // Next, succeed all and confirm output - complete(taskSets.last, Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) - complete(taskSets.last, Seq((Success, makeMapStatus("hostD", 1)))) - complete(taskSets.last, Seq((Success, 42))) - assert(results === Map(0 -> 42)) + val stage1Attempt6 = taskSets.last + val completions_1_6 = makeCompletions(stage1Attempt6) + complete(stage1Attempt6, completions_1_6) + + val stage2Attempt2 = taskSets.last + complete(stage2Attempt2, Seq((Success, 42))) + assertDataStructuresEmpty() + sc.listenerBus.waitUntilEmpty(1000) + assert(ended === true) + assert(results === Map(2 -> 42)) } test("trivial shuffle with multiple fetch failures") { @@ -1123,7 +1126,7 @@ class DAGSchedulerSuite submit(finalRdd, Array(0)) cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) - // complete stage 2 + // complete stage 0 complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) @@ -1131,7 +1134,7 @@ class DAGSchedulerSuite complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - // pretend stage 0 failed because hostA went down + // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: From b66d74e14f878ddee96270ed38a17abc7741be2f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Jul 2015 10:18:26 -0700 Subject: [PATCH 28/45] Added stage ID checks in most places and fixed naming for attempts to be 0-indexed --- .../spark/scheduler/DAGSchedulerSuite.scala | 162 +++++++++--------- 1 file changed, 77 insertions(+), 85 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 242bd4e695cd0..4b57c7fa21dea 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -479,31 +479,38 @@ class DAGSchedulerSuite assert(stageAttempt.stageAttemptId == attempt) } - def makeCompletions(stageAttempt: TaskSet): Seq[(Success.type, MapStatus)] = { + def makeCompletions(stageAttempt: TaskSet, reduceParts: Int): Seq[(Success.type, MapStatus)] = { stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, stageAttempt.tasks.size)) + (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.") { - // 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()) + setupStageAbortTest(sc) val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) @@ -514,15 +521,10 @@ class DAGSchedulerSuite for (attempt <- 0 until 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 next attempt for stage 0 checkStageId(0, attempt, stage0Attempt) - // Make each task in stage 0 success - val completions = makeCompletions(stage0Attempt) - // Run stage 0 - complete(stage0Attempt, completions) + 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 @@ -561,6 +563,8 @@ class DAGSchedulerSuite * aborted. */ test("Failures in different stages should not trigger an overall abort") { + setupStageAbortTest(sc) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId @@ -572,16 +576,11 @@ class DAGSchedulerSuite for (attempt <- 0 until 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 next attempt for stage 0 checkStageId(0, attempt, stage0Attempt) if (attempt < Stage.MAX_STAGE_FAILURES/2) { - // Make each task in stage 0 success - val completions = makeCompletions(stage0Attempt) - // Run stage 0 - complete(stage0Attempt, completions) + 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 @@ -597,7 +596,6 @@ class DAGSchedulerSuite ) } 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)) @@ -610,10 +608,11 @@ class DAGSchedulerSuite scheduler.resubmitFailedStages() } - val stage0Attempt = taskSets.last - val completions = stage0Attempt.tasks.zipWithIndex.map{ case (task, idx) => + val stage0Attempt4 = taskSets.last + val completions = stage0Attempt4.tasks.zipWithIndex.map{ case (task, idx) => (Success, makeMapStatus("host" + ('A' + idx).toChar, 2)) }.toSeq + checkStageId(0, 4, stage0Attempt4) // Complete first task complete(taskSets.last, completions) @@ -631,19 +630,7 @@ class DAGSchedulerSuite * their own, but only when the stage fails enough times. */ test("Multiple task failures in same stage should not abort the stage.") { - // 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()) + setupStageAbortTest(sc) val parts = 8 val shuffleMapRdd = new MyRDD(sc, parts, Nil) @@ -652,19 +639,19 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, parts, List(shuffleDep)) submit(reduceRdd, (0 until parts).toArray) - val stage0Attempt = taskSets.last - + val stage0Attempt0 = taskSets.last + checkStageId(0, 0, stage0Attempt0) // Make each task in stage 0 success, then fail all of stage 1 - val completions = makeCompletions(stage0Attempt) - complete(stage0Attempt, completions) + complete(stage0Attempt0, makeCompletions(stage0Attempt0, parts)) - val stage1Attempt = taskSets.last - val failures = stage1Attempt.tasks.zipWithIndex.map{ case (task, idx) => + 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, this time with all fetchs failing - complete(stage1Attempt, failures) + // Run Stage 1 with all fetchs failing + complete(stage1Attempt0, failures) // Resubmit and confirm that now all is well scheduler.resubmitFailedStages() @@ -673,13 +660,17 @@ class DAGSchedulerSuite assert(!ended) // Confirm job finished succesfully - val stage1Attempt2 = taskSets.last - val completions_1_2 = stage1Attempt2.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq + 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) - complete(stage1Attempt2, completions_1_2) sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) - assert(results === Map(1 -> 42)) + assert(results === (0 until parts).map{idx => idx -> 42}.toMap) } /** @@ -689,19 +680,7 @@ class DAGSchedulerSuite * but that doesn't matter, since they have successes in the middle. */ test("Abort should only trigger after consecutive stage failures") { - // 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()) + setupStageAbortTest(sc) val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache() val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) @@ -716,15 +695,16 @@ class DAGSchedulerSuite // Make each task in stage 0 success val stage0Attempt = taskSets.last - - val completions = makeCompletions(stage0Attempt); + checkStageId(0, attempt, stage0Attempt) // Run stage 0 - complete(stage0Attempt, completions) + 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, stage0Attempt) + val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} @@ -742,27 +722,38 @@ class DAGSchedulerSuite assert(!ended) } - // Now succeed stage 1 and fail stage 2 - val stage1attempt4 = taskSets.last - val completions_1_4 = makeCompletions(stage1attempt4) + // 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(stage1attempt4, completions_1_4) + complete(stage1Attempt3, makeCompletions(stage1Attempt3, 1)) // Fail stage 2 - complete(taskSets.last, Seq( + 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 stage1attempt5 = taskSets.last + val stage1attempt4 = taskSets.last + checkStageId(1, 4, stage1attempt4) val stage1Successes = - stage1attempt5.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} + stage1attempt4.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} // Run Stage 1, this time with a task failure - complete(stage1attempt5, + complete(stage1attempt4, Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepOne.shuffleId, 0, 0, "ignored"), null) ) ++ stage1Successes @@ -776,17 +767,18 @@ class DAGSchedulerSuite assert(!ended) // Next, succeed all and confirm output - val stage1Attempt6 = taskSets.last - val completions_1_6 = makeCompletions(stage1Attempt6) - complete(stage1Attempt6, completions_1_6) + val stage1Attempt5 = taskSets.last + checkStageId(1, 5, stage1Attempt5) + complete(stage1Attempt5, makeCompletions(stage1Attempt5, 1)) - val stage2Attempt2 = taskSets.last - complete(stage2Attempt2, Seq((Success, 42))) + val stage2Attempt1 = taskSets.last + checkStageId(2, 1, stage2Attempt1) + complete(stage2Attempt1, Seq((Success, 42))) assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) - assert(results === Map(2 -> 42)) + assert(results === Map(0 -> 42)) } test("trivial shuffle with multiple fetch failures") { From 2e058ba65998316661ab5923ddb6354f268a6bda Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Jul 2015 10:28:08 -0700 Subject: [PATCH 29/45] All tests passing. Still need to refactor multiple fetch failures per stage into a three task test --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 4b57c7fa21dea..ecdd36312f39e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -690,9 +690,6 @@ class DAGSchedulerSuite submit(finalRdd, Array(0)) for (attempt <- 0 until Stage.MAX_STAGE_FAILURES-1) { - println(s"attempt = $attempt") - println(taskSets.mkString(",")) - // Make each task in stage 0 success val stage0Attempt = taskSets.last checkStageId(0, attempt, stage0Attempt) @@ -703,7 +700,7 @@ class DAGSchedulerSuite // 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, stage0Attempt) + checkStageId(1, attempt, stage1Attempt) val stage1Successes = stage1Attempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", 1))} @@ -767,6 +764,11 @@ class DAGSchedulerSuite assert(!ended) // Next, succeed all and confirm output + // Rerun stage 0 + val stage0Attempt5 = taskSets.last + checkStageId(0, 5, stage0Attempt5) + complete(stage0Attempt5, makeCompletions(stage0Attempt5, 2)) + val stage1Attempt5 = taskSets.last checkStageId(1, 5, stage1Attempt5) complete(stage1Attempt5, makeCompletions(stage1Attempt5, 1)) From f79011b1979862db4a8fe7fcb6688e753c3b4f4a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Jul 2015 11:18:03 -0700 Subject: [PATCH 30/45] Updated remaining test for sequential test failures to have three stages to avoid introducing a fetch failure in stage 0. --- .../spark/scheduler/DAGSchedulerSuite.scala | 62 +++++++++++-------- 1 file changed, 36 insertions(+), 26 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ecdd36312f39e..10b070071b0c7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -565,42 +565,49 @@ class DAGSchedulerSuite test("Failures in different stages should not trigger an overall abort") { 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)) + 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 0 fails. + // 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) { - // 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)} + 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"), shuffleId, 0, 0, "ignored"), null)) - ++ stage1Successes + Seq((FetchFailed(makeBlockManagerId("hostA"), + shuffleDepOne.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 - ) + // 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 @@ -608,20 +615,23 @@ class DAGSchedulerSuite scheduler.resubmitFailedStages() } + // Complete all three stages succesfully val stage0Attempt4 = taskSets.last - val completions = stage0Attempt4.tasks.zipWithIndex.map{ case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, 2)) - }.toSeq checkStageId(0, 4, stage0Attempt4) + complete(stage0Attempt4, makeCompletions(stage0Attempt4, 2)) - // Complete first task - complete(taskSets.last, completions) + val stage1Attempt4 = taskSets.last + checkStageId(1, 4, stage1Attempt4) + complete(stage1Attempt4, makeCompletions(stage1Attempt4, 1)) - // Complete second task - complete(taskSets.last, Seq((Success, 42))) + 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(1 -> 42, 0 -> 42)) + assert(results === Map(0 -> 42)) } /** From 62532fa9abbb26ba55455d0b9d72d84fd9c1638a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Jul 2015 12:41:39 -0700 Subject: [PATCH 31/45] Nit fixes --- .../apache/spark/scheduler/DAGScheduler.scala | 4 ++-- .../scala/org/apache/spark/scheduler/Stage.scala | 7 ++++--- .../spark/scheduler/DAGSchedulerSuite.scala | 16 +++++++--------- 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 3a6d240b73a2e..35bec69dfa098 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1066,8 +1066,8 @@ class DAGScheduler( if (disallowStageRetryForTest) { abortStage(failedStage, "Fetch failure will not retry stage due to testing config") - } else if (failedStage.failAndShouldAbort(task)) { - abortStage(failedStage, s"Stage ${failedStage.name} " + + } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { + abortStage(failedStage, s"$failedStage (${failedStage.name}) " + s"has failed the maximum allowable number of times: ${Stage.MAX_STAGE_FAILURES}. " + s"Most recent failure reason: ${failureMessage}") } else if (failedStages.isEmpty) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 0bc90ed786125..01115dbed60e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -95,12 +95,13 @@ private[scheduler] abstract class Stage( } /** - * Check whether we should abort the failedStage due to multiple failures. + * Check whether we should abort the failedStage due to multiple consecutive fetch failures. + * * This method updates the running set of failed stage attempts and returns * true if the number of failures exceeds the allowable number of failures. */ - private[scheduler] def failAndShouldAbort(task: Task[_]): Boolean = { - attemptsFailedFromFetch.add(task.stageAttemptId) + private[scheduler] def failedOnFetchAndShouldAbort(stageAttemptId: Int): Boolean = { + attemptsFailedFromFetch.add(stageAttemptId) attemptsFailedFromFetch.size >= Stage.MAX_STAGE_FAILURES } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 10b070071b0c7..20b807a667bed 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -558,7 +558,7 @@ class DAGSchedulerSuite /** * 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 + * 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. */ @@ -624,20 +624,18 @@ class DAGSchedulerSuite 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)) + 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 do not trigger an abort on - * their own, but only when the stage fails enough times. + * 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 task failures in same stage should not abort the stage.") { setupStageAbortTest(sc) @@ -660,7 +658,7 @@ class DAGSchedulerSuite (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, idx, "ignored"), null) }.toSeq - // Run Stage 1 with all fetchs failing + // Run Stage 1 with all fetches failing complete(stage1Attempt0, failures) // Resubmit and confirm that now all is well @@ -681,6 +679,7 @@ class DAGSchedulerSuite sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) assert(results === (0 until parts).map{idx => idx -> 42}.toMap) + assertDataStructuresEmpty() } /** @@ -766,9 +765,8 @@ class DAGSchedulerSuite ) ++ stage1Successes ) - Thread.sleep(500) scheduler.resubmitFailedStages() - Thread.sleep(500) + // Confirm we have not yet aborted assert(scheduler.runningStages.nonEmpty) assert(!ended) From 1c1cb72f4ab82c34a85e563c0b5f277740fb8a51 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Jul 2015 13:52:02 -0700 Subject: [PATCH 32/45] Style --- core/src/main/scala/org/apache/spark/scheduler/Stage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 01115dbed60e5..a546639234e04 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -96,7 +96,7 @@ private[scheduler] abstract class Stage( /** * Check whether we should abort the failedStage due to multiple consecutive fetch failures. - * + * * This method updates the running set of failed stage attempts and returns * true if the number of failures exceeds the allowable number of failures. */ From 17e85deed0085822283631244839d6f110efccef Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 29 Jul 2015 14:44:34 -0700 Subject: [PATCH 33/45] Naming --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 20b807a667bed..92d3df2babdc1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -637,7 +637,7 @@ class DAGSchedulerSuite * 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 task failures in same stage should not abort the stage.") { + test("Multiple tasks w/ fetch failures in same stage should not abort the stage.") { setupStageAbortTest(sc) val parts = 8 From cf948508f077f379761ef3da0eb3bb295154c229 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 4 Aug 2015 14:33:05 -0700 Subject: [PATCH 34/45] refactored tests to eliminate reused code --- .../org/apache/spark/scheduler/Stage.scala | 5 +- .../spark/scheduler/DAGSchedulerSuite.scala | 230 ++++++++---------- 2 files changed, 106 insertions(+), 129 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index a546639234e04..da41ea9f4935f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,11 +17,12 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashSet + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.util.CallSite -import scala.collection.mutable.HashSet /** * A stage is a set of independent tasks all computing the same function that need to run as part @@ -121,7 +122,7 @@ private[scheduler] abstract class Stage( } } -private[spark] object Stage { +private[scheduler] object Stage { // The maximum number of times to retry a stage before aborting private[scheduler] val MAX_STAGE_FAILURES = 4 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 92d3df2babdc1..e4a0cbf667595 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -504,6 +504,74 @@ class DAGSchedulerSuite } } + // 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 numPartitions - The number of partitions in the next stage + */ + def completeNextShuffleMapSuccesfully(stageId: Int, attemptIdx: Int, numPartitions: Int): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + complete(stageAttempt, makeCompletions(stageAttempt, numPartitions)) + } + + /** + * Common code to get the next stage attempt, confirm it's the one we expect, and complete it + * with an intermediate FetchFailure. + * + * @param stageId - The current stageId + * @param attemptIdx - The current attempt count + * @param shuffleId - The shuffleId of the stage with a fetch failure + */ + def completeNextShuffleMapWithFailureAndSuccess(stageId: Int, + attemptIdx: Int, + shuffleId: Int, + numPartitions: Int): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + + val stageSuccesses = stageAttempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", + numPartitions))} + complete(stageAttempt, + Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) + ++ stageSuccesses + ) + } + + /** + * 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 shuffleId - The shuffleId of the stage with a fetch failure + */ + def completeNextShuffleMapWithFetchFailure(stageId: Int, attemptIdx: Int, shuffleId: Int): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + + complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map{ case (task, idx) => + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, idx, "ignored"), null) + }.toSeq) + } + + /** + * Common code to get the next 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 completeNextShuffleMapWithSuccess_42 (stageId: Int, attemptIdx: Int): Unit = { + val stageAttempt = taskSets.last + checkStageId(stageId, attemptIdx, stageAttempt) + 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 @@ -520,24 +588,11 @@ class DAGSchedulerSuite 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)) + completeNextShuffleMapSuccesfully(0, attempt, numPartitions = 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 - ) + completeNextShuffleMapWithFailureAndSuccess(1, attempt, shuffleId, 2) // 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 @@ -575,39 +630,23 @@ class DAGSchedulerSuite // 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) { + println(s"attempt = $attempt") + println(taskSets.mkString(",")) + // 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)) + completeNextShuffleMapSuccesfully(0, attempt, numPartitions = 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 - ) + completeNextShuffleMapWithFailureAndSuccess(1, attempt, shuffleDepOne.shuffleId, + numPartitions = 1) } else { - // Run stage 1 - val stage1Attempt = taskSets.last - checkStageId(1, attempt, stage1Attempt) - complete(stage1Attempt, makeCompletions(stage1Attempt, 1)) + completeNextShuffleMapSuccesfully(1, attempt, numPartitions = 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))) + completeNextShuffleMapWithFetchFailure(2, attempt - Stage.MAX_STAGE_FAILURES / 2, + shuffleDepTwo.shuffleId) } // this will (potentially) trigger a resubmission of stage 0, since we've lost some of its @@ -615,18 +654,11 @@ class DAGSchedulerSuite scheduler.resubmitFailedStages() } - // Complete all three stages succesfully - val stage0Attempt4 = taskSets.last - checkStageId(0, 4, stage0Attempt4) - complete(stage0Attempt4, makeCompletions(stage0Attempt4, 2)) + completeNextShuffleMapSuccesfully(0, 4, numPartitions = 2) + completeNextShuffleMapSuccesfully(1, 4, numPartitions = 1) - val stage1Attempt4 = taskSets.last - checkStageId(1, 4, stage1Attempt4) - complete(stage1Attempt4, makeCompletions(stage1Attempt4, 1)) - - val stage2Attempt = taskSets.last - checkStageId(2, Stage.MAX_STAGE_FAILURES/2, stage2Attempt) - complete(stage2Attempt, Seq((Success, 42))) + // Succeed stage2 with a "42" + completeNextShuffleMapWithSuccess_42(2, Stage.MAX_STAGE_FAILURES/2) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -647,19 +679,9 @@ class DAGSchedulerSuite 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)) + completeNextShuffleMapSuccesfully(0, 0, numPartitions = 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 fetches failing - complete(stage1Attempt0, failures) + completeNextShuffleMapWithFetchFailure(1, 0, shuffleId) // Resubmit and confirm that now all is well scheduler.resubmitFailedStages() @@ -667,15 +689,11 @@ class DAGSchedulerSuite 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) + // Complete stage 0 and then stage 1 with a "42" + completeNextShuffleMapSuccesfully(0, 1, numPartitions = parts) + completeNextShuffleMapWithSuccess_42(1, 1) + // Confirm job finished succesfully sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) assert(results === (0 until parts).map{idx => idx -> 42}.toMap) @@ -700,26 +718,12 @@ class DAGSchedulerSuite 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)) + completeNextShuffleMapSuccesfully(0, attempt, numPartitions = 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 - ) + completeNextShuffleMapWithFailureAndSuccess(1, attempt, shuffleDepOne.shuffleId, + numPartitions = 1) scheduler.resubmitFailedStages() @@ -728,42 +732,20 @@ class DAGSchedulerSuite 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)) + // Rerun stage 0 and 1 + completeNextShuffleMapSuccesfully(0, 3, numPartitions = 2) + completeNextShuffleMapSuccesfully(1, 3, numPartitions = 1) // Fail stage 2 - val stage2Attempt0 = taskSets.last - checkStageId(2, 0, stage2Attempt0) - complete(stage2Attempt0, Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + completeNextShuffleMapWithFetchFailure(2, 0, shuffleDepTwo.shuffleId) scheduler.resubmitFailedStages() // Rerun stage 0 - val stage0Attempt4 = taskSets.last - checkStageId(0, 4, stage0Attempt4) - complete(stage0Attempt4, makeCompletions(stage0Attempt4, 2)) + completeNextShuffleMapSuccesfully(0, 4, numPartitions = 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 - ) + completeNextShuffleMapWithFailureAndSuccess(1, 4, shuffleDepOne.shuffleId, numPartitions = 1) scheduler.resubmitFailedStages() @@ -772,18 +754,12 @@ class DAGSchedulerSuite assert(!ended) // Next, succeed all and confirm output - // Rerun stage 0 - val stage0Attempt5 = taskSets.last - checkStageId(0, 5, stage0Attempt5) - complete(stage0Attempt5, makeCompletions(stage0Attempt5, 2)) - - val stage1Attempt5 = taskSets.last - checkStageId(1, 5, stage1Attempt5) - complete(stage1Attempt5, makeCompletions(stage1Attempt5, 1)) + // Rerun stage 0 + 1 + completeNextShuffleMapSuccesfully(0, 5, numPartitions = 2) + completeNextShuffleMapSuccesfully(1, 5, numPartitions = 1) - val stage2Attempt1 = taskSets.last - checkStageId(2, 1, stage2Attempt1) - complete(stage2Attempt1, Seq((Success, 42))) + // Succeed stage 2 and verify results + completeNextShuffleMapWithSuccess_42(2, 1) assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(1000) From 7c6f60f2ffd6ccbf23833c2d19497da488a084d5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 4 Aug 2015 14:44:55 -0700 Subject: [PATCH 35/45] Style --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index e4a0cbf667595..5fa696c1dc036 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -550,7 +550,9 @@ class DAGSchedulerSuite * @param attemptIdx - The current attempt count * @param shuffleId - The shuffleId of the stage with a fetch failure */ - def completeNextShuffleMapWithFetchFailure(stageId: Int, attemptIdx: Int, shuffleId: Int): Unit = { + def completeNextShuffleMapWithFetchFailure(stageId: Int, + attemptIdx: Int, + shuffleId: Int): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) @@ -630,9 +632,6 @@ class DAGSchedulerSuite // 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) { - println(s"attempt = $attempt") - println(taskSets.mkString(",")) - // Complete all the tasks for the current attempt of stage 0 successfully completeNextShuffleMapSuccesfully(0, attempt, numPartitions = 2) From 13af9701540b1f60fb758bc4b02dcea452f4652f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 5 Aug 2015 14:24:15 -0700 Subject: [PATCH 36/45] Nit fixes. --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../org/apache/spark/scheduler/Stage.scala | 5 +- .../spark/scheduler/DAGSchedulerSuite.scala | 88 ++++++++++--------- 3 files changed, 50 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 35bec69dfa098..3108a90779e5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1196,8 +1196,8 @@ class DAGScheduler( // Clear failure count for this stage, now that it's succeeded. // We only limit consecutive failures of stage attempts,so that if a stage is - // re-used many times in a long-running job, unrelated failures that are spaced out in time - // don't eventually cause the stage to be aborted. + // re-used many times in a long-running job, unrelated failures don't eventually cause the + // stage to be aborted. stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index da41ea9f4935f..33bd3f418c636 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -23,7 +23,6 @@ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.util.CallSite - /** * A stage is a set of independent tasks all computing the same function that need to run as part * of a Spark job, where all the tasks have the same shuffle dependencies. Each DAG of tasks run @@ -87,7 +86,7 @@ private[scheduler] abstract class Stage( * B) A failure has not yet been registered for this stage attempt. There may be multiple * concurrent failures for a sinlge stage since we may have multiple tasks executing at the same * time, one or many of which may fail. Also, even though there may only be one non-zombie stage - * attemp, zombie stages may still have running tasks. + * attempt, zombie stage attempts may still have running tasks. */ private val attemptsFailedFromFetch = new HashSet[Int] @@ -124,5 +123,5 @@ private[scheduler] abstract class Stage( private[scheduler] object Stage { // The maximum number of times to retry a stage before aborting - private[scheduler] val MAX_STAGE_FAILURES = 4 + val MAX_STAGE_FAILURES = 4 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 5fa696c1dc036..741fb255425ea 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -511,12 +511,13 @@ class DAGSchedulerSuite * * @param stageId - The current stageId * @param attemptIdx - The current attempt count - * @param numPartitions - The number of partitions in the next stage + * @param numShufflePartitions - The number of partitions in the next stage */ - def completeNextShuffleMapSuccesfully(stageId: Int, attemptIdx: Int, numPartitions: Int): Unit = { + def completeNextShuffleMapSuccesfully(stageId: Int, attemptIdx: Int, + numShufflePartitions: Int): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) - complete(stageAttempt, makeCompletions(stageAttempt, numPartitions)) + complete(stageAttempt, makeCompletions(stageAttempt, numShufflePartitions)) } /** @@ -525,11 +526,11 @@ class DAGSchedulerSuite * * @param stageId - The current stageId * @param attemptIdx - The current attempt count - * @param shuffleId - The shuffleId of the stage with a fetch failure + * @param shuffleDep - The shuffle dependency of the stage with a fetch failure */ def completeNextShuffleMapWithFailureAndSuccess(stageId: Int, attemptIdx: Int, - shuffleId: Int, + shuffleDep: ShuffleDependency[_,_,_], numPartitions: Int): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) @@ -537,7 +538,7 @@ class DAGSchedulerSuite val stageSuccesses = stageAttempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", numPartitions))} complete(stageAttempt, - Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) + Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, 0, "ignored"), null)) ++ stageSuccesses ) } @@ -548,38 +549,39 @@ class DAGSchedulerSuite * * @param stageId - The current stageId * @param attemptIdx - The current attempt count - * @param shuffleId - The shuffleId of the stage with a fetch failure + * @param shuffleDep - The shuffle dependency of the stage with a fetch failure */ - def completeNextShuffleMapWithFetchFailure(stageId: Int, + def completeNextStageWithFetchFailure(stageId: Int, attemptIdx: Int, - shuffleId: Int): Unit = { + shuffleDep: ShuffleDependency[_,_,_]): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map{ case (task, idx) => - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, idx, "ignored"), null) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) }.toSeq) } /** - * Common code to get the next stage attempt, confirm it's the one we expect, and complete it - * with a success where we return 42. + * 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 completeNextShuffleMapWithSuccess_42 (stageId: Int, attemptIdx: Int): Unit = { + 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 stage abort to avoid endless retries. + * trigger an overall job abort to avoid endless retries. */ - test("Multiple consecutive stage failures should lead to task being aborted.") { + test("Multiple consecutive stage failures should lead to job being aborted.") { setupStageAbortTest(sc) val shuffleMapRdd = new MyRDD(sc, 2, Nil) @@ -590,11 +592,11 @@ class DAGSchedulerSuite for (attempt <- 0 until Stage.MAX_STAGE_FAILURES) { // Complete all the tasks for the current attempt of stage 0 successfully - completeNextShuffleMapSuccesfully(0, attempt, numPartitions = 2) + 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 - completeNextShuffleMapWithFailureAndSuccess(1, attempt, shuffleId, 2) + 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 @@ -608,7 +610,11 @@ class DAGSchedulerSuite assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(1000) assert(ended) - assert(jobResult.isInstanceOf[JobFailed]) + jobResult match { + case JobFailed(reason) => + assert(reason.getMessage.contains("ResultStage 1 () has failed the maximum")) + case other => fail(s"expected JobFailed, not $other") + } } } } @@ -633,19 +639,18 @@ class DAGSchedulerSuite // 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, numPartitions = 2) + 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 - completeNextShuffleMapWithFailureAndSuccess(1, attempt, shuffleDepOne.shuffleId, - numPartitions = 1) + completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) } else { - completeNextShuffleMapSuccesfully(1, attempt, numPartitions = 1) + completeNextShuffleMapSuccesfully(1, attempt, numShufflePartitions = 1) // Fail stage 2 - completeNextShuffleMapWithFetchFailure(2, attempt - Stage.MAX_STAGE_FAILURES / 2, - shuffleDepTwo.shuffleId) + 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 @@ -653,11 +658,11 @@ class DAGSchedulerSuite scheduler.resubmitFailedStages() } - completeNextShuffleMapSuccesfully(0, 4, numPartitions = 2) - completeNextShuffleMapSuccesfully(1, 4, numPartitions = 1) + completeNextShuffleMapSuccesfully(0, 4, numShufflePartitions = 2) + completeNextShuffleMapSuccesfully(1, 4, numShufflePartitions = 1) // Succeed stage2 with a "42" - completeNextShuffleMapWithSuccess_42(2, Stage.MAX_STAGE_FAILURES/2) + completeNextResultStageWithSuccess(2, Stage.MAX_STAGE_FAILURES/2) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -678,9 +683,9 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, parts, List(shuffleDep)) submit(reduceRdd, (0 until parts).toArray) - completeNextShuffleMapSuccesfully(0, 0, numPartitions = parts) + completeNextShuffleMapSuccesfully(0, 0, numShufflePartitions = parts) - completeNextShuffleMapWithFetchFailure(1, 0, shuffleId) + completeNextStageWithFetchFailure(1, 0, shuffleDep) // Resubmit and confirm that now all is well scheduler.resubmitFailedStages() @@ -689,8 +694,8 @@ class DAGSchedulerSuite assert(!ended) // Complete stage 0 and then stage 1 with a "42" - completeNextShuffleMapSuccesfully(0, 1, numPartitions = parts) - completeNextShuffleMapWithSuccess_42(1, 1) + completeNextShuffleMapSuccesfully(0, 1, numShufflePartitions = parts) + completeNextResultStageWithSuccess(1, 1) // Confirm job finished succesfully sc.listenerBus.waitUntilEmpty(1000) @@ -717,12 +722,11 @@ class DAGSchedulerSuite for (attempt <- 0 until Stage.MAX_STAGE_FAILURES-1) { // Make each task in stage 0 success - completeNextShuffleMapSuccesfully(0, attempt, numPartitions = 2) + 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 - completeNextShuffleMapWithFailureAndSuccess(1, attempt, shuffleDepOne.shuffleId, - numPartitions = 1) + completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) scheduler.resubmitFailedStages() @@ -732,19 +736,19 @@ class DAGSchedulerSuite } // Rerun stage 0 and 1 - completeNextShuffleMapSuccesfully(0, 3, numPartitions = 2) - completeNextShuffleMapSuccesfully(1, 3, numPartitions = 1) + completeNextShuffleMapSuccesfully(0, 3, numShufflePartitions = 2) + completeNextShuffleMapSuccesfully(1, 3, numShufflePartitions = 1) // Fail stage 2 - completeNextShuffleMapWithFetchFailure(2, 0, shuffleDepTwo.shuffleId) + completeNextStageWithFetchFailure(2, 0, shuffleDepTwo) scheduler.resubmitFailedStages() // Rerun stage 0 - completeNextShuffleMapSuccesfully(0, 4, numPartitions = 2) + completeNextShuffleMapSuccesfully(0, 4, numShufflePartitions = 2) // Now again, fail stage 1 (up to MAX_FAILURES) but confirm that this doesn't trigger an abort // since we succeeded in between - completeNextShuffleMapWithFailureAndSuccess(1, 4, shuffleDepOne.shuffleId, numPartitions = 1) + completeNextStageWithFetchFailure(1, 4, shuffleDepOne) scheduler.resubmitFailedStages() @@ -754,11 +758,11 @@ class DAGSchedulerSuite // Next, succeed all and confirm output // Rerun stage 0 + 1 - completeNextShuffleMapSuccesfully(0, 5, numPartitions = 2) - completeNextShuffleMapSuccesfully(1, 5, numPartitions = 1) + completeNextShuffleMapSuccesfully(0, 5, numShufflePartitions = 2) + completeNextShuffleMapSuccesfully(1, 5, numShufflePartitions = 1) // Succeed stage 2 and verify results - completeNextShuffleMapWithSuccess_42(2, 1) + completeNextResultStageWithSuccess(2, 1) assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(1000) From 09929da64052d50b3a55141d5cf2fd144c52ea62 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 5 Aug 2015 14:34:44 -0700 Subject: [PATCH 37/45] Style --- .../spark/scheduler/DAGSchedulerSuite.scala | 23 ------------------- 1 file changed, 23 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 741fb255425ea..8763178ed97da 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -520,29 +520,6 @@ class DAGSchedulerSuite complete(stageAttempt, makeCompletions(stageAttempt, numShufflePartitions)) } - /** - * Common code to get the next stage attempt, confirm it's the one we expect, and complete it - * with an intermediate 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 completeNextShuffleMapWithFailureAndSuccess(stageId: Int, - attemptIdx: Int, - shuffleDep: ShuffleDependency[_,_,_], - numPartitions: Int): Unit = { - val stageAttempt = taskSets.last - checkStageId(stageId, attemptIdx, stageAttempt) - - val stageSuccesses = stageAttempt.tasks.tail.map { _ => (Success, makeMapStatus("hostB", - numPartitions))} - complete(stageAttempt, - Seq((FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, 0, "ignored"), null)) - ++ stageSuccesses - ) - } - /** * Common code to get the next stage attempt, confirm it's the one we expect, and complete it * with all FetchFailure. From eb15503bbd7693c46ae4c1659fd267e05d2045c6 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 5 Aug 2015 15:31:52 -0700 Subject: [PATCH 38/45] Style --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 8763178ed97da..7a9c9d4812181 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -530,7 +530,7 @@ class DAGSchedulerSuite */ def completeNextStageWithFetchFailure(stageId: Int, attemptIdx: Int, - shuffleDep: ShuffleDependency[_,_,_]): Unit = { + shuffleDep: ShuffleDependency[_, _, _]): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) From 01d6841e20cda21f2fab09ec495fd3ee67aa1b0d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 6 Aug 2015 08:09:56 -0700 Subject: [PATCH 39/45] Nits --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7a9c9d4812181..001099cdb4b5c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -572,7 +572,7 @@ class DAGSchedulerSuite 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 + // Fail all these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDep) // this will (potentially) trigger a resubmission of stage 0, since we've lost some of its @@ -620,7 +620,7 @@ class DAGSchedulerSuite 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 + // Fail all these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) } else { completeNextShuffleMapSuccesfully(1, attempt, numShufflePartitions = 1) @@ -650,7 +650,7 @@ class DAGSchedulerSuite * 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.") { + test("Multiple tasks w/ fetch failures in same stage attempt should not abort the stage.") { setupStageAbortTest(sc) val parts = 8 @@ -702,7 +702,7 @@ class DAGSchedulerSuite 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 + // Fail these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) scheduler.resubmitFailedStages() From 4da18a1acd2f0ab8d999da145415ab57cec09c37 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 19 Aug 2015 11:28:27 -0700 Subject: [PATCH 40/45] Style fix --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 281cf968ef1e6..40b0e40bdc115 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1101,7 +1101,8 @@ class DAGScheduler( } if (disallowStageRetryForTest) { - abortStage(failedStage, "Fetch failure will not retry stage due to testing config", None) + abortStage(failedStage, "Fetch failure will not retry stage due to testing config", + None) } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { abortStage(failedStage, s"$failedStage (${failedStage.name}) " + s"has failed the maximum allowable number of times: ${Stage.MAX_STAGE_FAILURES}. " + From f928ff3ec4dcf56db10a2d7bb6632db8b61fc866 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 1 Sep 2015 13:02:33 -0700 Subject: [PATCH 41/45] Updated PR description and minor nits --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 618d4320740bd..76064548cfd10 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -513,7 +513,9 @@ class DAGSchedulerSuite * @param attemptIdx - The current attempt count * @param numShufflePartitions - The number of partitions in the next stage */ - def completeNextShuffleMapSuccesfully(stageId: Int, attemptIdx: Int, + def completeNextShuffleMapSuccesfully( + stageId: Int, + attemptIdx: Int, numShufflePartitions: Int): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) @@ -528,7 +530,8 @@ class DAGSchedulerSuite * @param attemptIdx - The current attempt count * @param shuffleDep - The shuffle dependency of the stage with a fetch failure */ - def completeNextStageWithFetchFailure(stageId: Int, + def completeNextStageWithFetchFailure( + stageId: Int, attemptIdx: Int, shuffleDep: ShuffleDependency[_, _, _]): Unit = { val stageAttempt = taskSets.last From e22ce7cf0848f3a9888ad61f89bc958ff9767bde Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 2 Sep 2015 10:42:49 -0700 Subject: [PATCH 42/45] Updated with feedback from PR --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/Stage.scala | 20 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 102 +++++++++--------- 3 files changed, 59 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9133533f2c08e..deaee4d629dfa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1118,7 +1118,7 @@ class DAGScheduler( None) } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { abortStage(failedStage, s"$failedStage (${failedStage.name}) " + - s"has failed the maximum allowable number of times: ${Stage.MAX_STAGE_FAILURES}. " + + s"has failed the maximum allowable number of times: ${Stage.MAX_CONSECUTIVE_FAILURES}. " + s"Most recent failure reason: ${failureMessage}", None) } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index ecce94100712e..32b285aa04cc9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -93,16 +93,10 @@ private[scheduler] abstract class Stage( private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId) /** - * Spark is resilient to executors dying by retrying stages on FetchFailures. Here, we keep track - * of unique stage failures (per stage attempt) triggered by fetch failures to prevent endless - * stage retries. Specifically, per stage we wish to only record a failure when the following - * holds: - * - * A) A fetch failure was observed - * B) A failure has not yet been registered for this stage attempt. There may be multiple - * concurrent failures for a sinlge stage since we may have multiple tasks executing at the same - * time, one or many of which may fail. Also, even though there may only be one non-zombie stage - * attempt, zombie stage attempts may still have running tasks. + * Set of IDs of stage attempts that have failed with a FetchFailure. We keep track of these + * failures in order to avoid endless retries if a stage keeps failing with a FetchFailure. + * We keep track of each attempt ID that has failed to avoid recording duplicate failures if + * multiple tasks from the same stage attempt fail. */ private val attemptsFailedFromFetch = new HashSet[Int] @@ -118,7 +112,7 @@ private[scheduler] abstract class Stage( */ private[scheduler] def failedOnFetchAndShouldAbort(stageAttemptId: Int): Boolean = { attemptsFailedFromFetch.add(stageAttemptId) - attemptsFailedFromFetch.size >= Stage.MAX_STAGE_FAILURES + attemptsFailedFromFetch.size >= Stage.MAX_CONSECUTIVE_FAILURES } /** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */ @@ -141,6 +135,6 @@ private[scheduler] abstract class Stage( } private[scheduler] object Stage { - // The maximum number of times to retry a stage before aborting - val MAX_STAGE_FAILURES = 4 + // The number of consecutive failures allowed before a stage is aborted + val MAX_CONSECUTIVE_FAILURES = 4 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 76064548cfd10..8c9133db4d60c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -473,6 +473,7 @@ 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) @@ -485,6 +486,7 @@ class DAGSchedulerSuite }.toSeq } + // Helper functions to extract commonly used code in Fetch Failure test cases def setupStageAbortTest(sc: SparkContext) { sc.listenerBus.addListener(new EndListener()) ended = false @@ -504,7 +506,6 @@ class DAGSchedulerSuite } } - // 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. @@ -556,6 +557,42 @@ class DAGSchedulerSuite 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("Multiple tasks w/ fetch failures in same stage attempt 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) + + completeNextShuffleMapSuccesfully(0, 0, numShufflePartitions = parts) + + completeNextStageWithFetchFailure(1, 0, shuffleDep) + + // Resubmit and confirm that now all is well + scheduler.resubmitFailedStages() + + assert(scheduler.runningStages.nonEmpty) + assert(!ended) + + // Complete stage 0 and then stage 1 with a "42" + completeNextShuffleMapSuccesfully(0, 1, numShufflePartitions = parts) + completeNextResultStageWithSuccess(1, 1) + + // Confirm job finished succesfully + sc.listenerBus.waitUntilEmpty(1000) + assert(ended === true) + assert(results === (0 until parts).map{idx => idx -> 42}.toMap) + assertDataStructuresEmpty() + } + /** * 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 @@ -570,7 +607,7 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - for (attempt <- 0 until Stage.MAX_STAGE_FAILURES) { + for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES) { // Complete all the tasks for the current attempt of stage 0 successfully completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 2) @@ -582,11 +619,11 @@ class DAGSchedulerSuite // map output, for the next iteration through the loop scheduler.resubmitFailedStages() - if (attempt < Stage.MAX_STAGE_FAILURES-1) { + if (attempt < Stage.MAX_CONSECUTIVE_FAILURES-1) { assert(scheduler.runningStages.nonEmpty) assert(!ended) } else { - // Stage has been aborted and removed from running stages + // Stage should have been aborted and removed from running stages assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(1000) assert(ended) @@ -600,10 +637,9 @@ class DAGSchedulerSuite } /** - * 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. + * In this test, we create a job with two consecutive shuffles, and simulate 2 failures for each + * shuffle fetch. In total 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) @@ -617,11 +653,11 @@ class DAGSchedulerSuite // 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) { + for (attempt <- 0 until Stage.MAX_CONSECUTIVE_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) { + if (attempt < Stage.MAX_CONSECUTIVE_FAILURES / 2) { // Now we should have a new taskSet, for a new attempt of stage 1. // Fail all these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) @@ -629,7 +665,7 @@ class DAGSchedulerSuite completeNextShuffleMapSuccesfully(1, attempt, numShufflePartitions = 1) // Fail stage 2 - completeNextStageWithFetchFailure(2, attempt - Stage.MAX_STAGE_FAILURES / 2, + completeNextStageWithFetchFailure(2, attempt - Stage.MAX_CONSECUTIVE_FAILURES / 2, shuffleDepTwo) } @@ -642,55 +678,19 @@ class DAGSchedulerSuite completeNextShuffleMapSuccesfully(1, 4, numShufflePartitions = 1) // Succeed stage2 with a "42" - completeNextResultStageWithSuccess(2, Stage.MAX_STAGE_FAILURES/2) + completeNextResultStageWithSuccess(2, Stage.MAX_CONSECUTIVE_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 attempt 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) - - completeNextShuffleMapSuccesfully(0, 0, numShufflePartitions = parts) - - completeNextStageWithFetchFailure(1, 0, shuffleDep) - - // Resubmit and confirm that now all is well - scheduler.resubmitFailedStages() - - assert(scheduler.runningStages.nonEmpty) - assert(!ended) - - // Complete stage 0 and then stage 1 with a "42" - completeNextShuffleMapSuccesfully(0, 1, numShufflePartitions = parts) - completeNextResultStageWithSuccess(1, 1) - - // Confirm job finished succesfully - sc.listenerBus.waitUntilEmpty(1000) - assert(ended === true) - assert(results === (0 until parts).map{idx => idx -> 42}.toMap) - assertDataStructuresEmpty() - } - /** * 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 + * fail multiple times, 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") { + test("Non-consecutive stage failures don't trigger abort") { setupStageAbortTest(sc) val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache() @@ -700,7 +700,7 @@ class DAGSchedulerSuite val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) - for (attempt <- 0 until Stage.MAX_STAGE_FAILURES-1) { + for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES-1) { // Make each task in stage 0 success completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 2) From 2bd4138645554e017b697cd9d846cabddaff4f83 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 2 Sep 2015 11:12:08 -0700 Subject: [PATCH 43/45] Style fix --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index deaee4d629dfa..5e1c968007ce4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1118,7 +1118,8 @@ class DAGScheduler( None) } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { abortStage(failedStage, s"$failedStage (${failedStage.name}) " + - s"has failed the maximum allowable number of times: ${Stage.MAX_CONSECUTIVE_FAILURES}. " + + s"has failed the maximum allowable number of " + + s"times: ${Stage.MAX_CONSECUTIVE_FAILURES}. " + s"Most recent failure reason: ${failureMessage}", None) } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because From 1d44e0c3da2c5be2f77df83121380b6e2a1037d1 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 2 Sep 2015 16:28:01 -0700 Subject: [PATCH 44/45] Test case updates and nit fixes --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/Stage.scala | 14 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 75 +++++++++---------- 3 files changed, 44 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5e1c968007ce4..d673cb0946639 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1119,7 +1119,7 @@ class DAGScheduler( } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { abortStage(failedStage, s"$failedStage (${failedStage.name}) " + s"has failed the maximum allowable number of " + - s"times: ${Stage.MAX_CONSECUTIVE_FAILURES}. " + + s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " + s"Most recent failure reason: ${failureMessage}", None) } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 32b285aa04cc9..c086535782c23 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -93,15 +93,15 @@ private[scheduler] abstract class Stage( private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId) /** - * Set of IDs of stage attempts that have failed with a FetchFailure. We keep track of these + * Set of stage attempt IDs that have failed with a FetchFailure. We keep track of these * failures in order to avoid endless retries if a stage keeps failing with a FetchFailure. * We keep track of each attempt ID that has failed to avoid recording duplicate failures if - * multiple tasks from the same stage attempt fail. + * multiple tasks from the same stage attempt fail (SPARK-5945). */ - private val attemptsFailedFromFetch = new HashSet[Int] + private val fetchFailedAttemptIds = new HashSet[Int] private[scheduler] def clearFailures() : Unit = { - attemptsFailedFromFetch.clear() + fetchFailedAttemptIds.clear() } /** @@ -111,8 +111,8 @@ private[scheduler] abstract class Stage( * true if the number of failures exceeds the allowable number of failures. */ private[scheduler] def failedOnFetchAndShouldAbort(stageAttemptId: Int): Boolean = { - attemptsFailedFromFetch.add(stageAttemptId) - attemptsFailedFromFetch.size >= Stage.MAX_CONSECUTIVE_FAILURES + fetchFailedAttemptIds.add(stageAttemptId) + fetchFailedAttemptIds.size >= Stage.MAX_CONSECUTIVE_FETCH_FAILURES } /** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */ @@ -136,5 +136,5 @@ private[scheduler] abstract class Stage( private[scheduler] object Stage { // The number of consecutive failures allowed before a stage is aborted - val MAX_CONSECUTIVE_FAILURES = 4 + val MAX_CONSECUTIVE_FETCH_FAILURES = 4 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 8c9133db4d60c..5cb7b81b29d68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -475,19 +475,14 @@ class DAGSchedulerSuite // Helper function to validate state when creating tests for task failures - def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) { + private 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 - } // Helper functions to extract commonly used code in Fetch Failure test cases - def setupStageAbortTest(sc: SparkContext) { + private def setupStageAbortTest(sc: SparkContext) { sc.listenerBus.addListener(new EndListener()) ended = false jobResult = null @@ -508,19 +503,22 @@ class DAGSchedulerSuite /** * Common code to get the next stage attempt, confirm it's the one we expect, and complete it - * succesfullly. + * successfully. * * @param stageId - The current stageId * @param attemptIdx - The current attempt count * @param numShufflePartitions - The number of partitions in the next stage */ - def completeNextShuffleMapSuccesfully( + private def completeShuffleMapStageSuccessfully( stageId: Int, attemptIdx: Int, numShufflePartitions: Int): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) - complete(stageAttempt, makeCompletions(stageAttempt, numShufflePartitions)) + complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { + case (task, idx) => + (Success, makeMapStatus("host" + ('A' + idx).toChar, numShufflePartitions)) + }.toSeq) } /** @@ -531,14 +529,13 @@ class DAGSchedulerSuite * @param attemptIdx - The current attempt count * @param shuffleDep - The shuffle dependency of the stage with a fetch failure */ - def completeNextStageWithFetchFailure( + 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) => + complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) }.toSeq) } @@ -550,7 +547,7 @@ class DAGSchedulerSuite * @param stageId - The current stageId * @param attemptIdx - The current attempt count */ - def completeNextResultStageWithSuccess (stageId: Int, attemptIdx: Int): Unit = { + private def completeNextResultStageWithSuccess(stageId: Int, attemptIdx: Int): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage]) @@ -562,7 +559,7 @@ class DAGSchedulerSuite * 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 attempt should not abort the stage.") { + test("Single fetch failure should not abort the stage.") { setupStageAbortTest(sc) val parts = 8 @@ -572,7 +569,7 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, parts, List(shuffleDep)) submit(reduceRdd, (0 until parts).toArray) - completeNextShuffleMapSuccesfully(0, 0, numShufflePartitions = parts) + completeShuffleMapStageSuccessfully(0, 0, numShufflePartitions = parts) completeNextStageWithFetchFailure(1, 0, shuffleDep) @@ -583,13 +580,13 @@ class DAGSchedulerSuite assert(!ended) // Complete stage 0 and then stage 1 with a "42" - completeNextShuffleMapSuccesfully(0, 1, numShufflePartitions = parts) + completeShuffleMapStageSuccessfully(0, 1, numShufflePartitions = parts) completeNextResultStageWithSuccess(1, 1) // Confirm job finished succesfully sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) - assert(results === (0 until parts).map{idx => idx -> 42}.toMap) + assert(results === (0 until parts).map { idx => idx -> 42 }.toMap) assertDataStructuresEmpty() } @@ -598,7 +595,7 @@ class DAGSchedulerSuite * 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.") { + test("Multiple consecutive stage fetch failures should lead to job being aborted.") { setupStageAbortTest(sc) val shuffleMapRdd = new MyRDD(sc, 2, Nil) @@ -607,19 +604,19 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES) { + for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES) { // Complete all the tasks for the current attempt of stage 0 successfully - completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) // Now we should have a new taskSet, for a new attempt of stage 1. // Fail all these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDep) - // this will (potentially) trigger a resubmission of stage 0, since we've lost some of its + // this will 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_CONSECUTIVE_FAILURES-1) { + if (attempt < Stage.MAX_CONSECUTIVE_FETCH_FAILURES - 1) { assert(scheduler.runningStages.nonEmpty) assert(!ended) } else { @@ -653,32 +650,32 @@ class DAGSchedulerSuite // 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_CONSECUTIVE_FAILURES) { + for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES) { // Complete all the tasks for the current attempt of stage 0 successfully - completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) - if (attempt < Stage.MAX_CONSECUTIVE_FAILURES / 2) { + if (attempt < Stage.MAX_CONSECUTIVE_FETCH_FAILURES / 2) { // Now we should have a new taskSet, for a new attempt of stage 1. // Fail all these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) } else { - completeNextShuffleMapSuccesfully(1, attempt, numShufflePartitions = 1) + completeShuffleMapStageSuccessfully(1, attempt, numShufflePartitions = 1) // Fail stage 2 - completeNextStageWithFetchFailure(2, attempt - Stage.MAX_CONSECUTIVE_FAILURES / 2, + completeNextStageWithFetchFailure(2, attempt - Stage.MAX_CONSECUTIVE_FETCH_FAILURES / 2, shuffleDepTwo) } - // this will (potentially) trigger a resubmission of stage 0, since we've lost some of its + // this will 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) + completeShuffleMapStageSuccessfully(0, 4, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(1, 4, numShufflePartitions = 1) // Succeed stage2 with a "42" - completeNextResultStageWithSuccess(2, Stage.MAX_CONSECUTIVE_FAILURES/2) + completeNextResultStageWithSuccess(2, Stage.MAX_CONSECUTIVE_FETCH_FAILURES/2) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -700,9 +697,9 @@ class DAGSchedulerSuite val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) - for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FAILURES-1) { + for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES - 1) { // Make each task in stage 0 success - completeNextShuffleMapSuccesfully(0, attempt, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) // Now we should have a new taskSet, for a new attempt of stage 1. // Fail these tasks with FetchFailure @@ -716,15 +713,15 @@ class DAGSchedulerSuite } // Rerun stage 0 and 1 - completeNextShuffleMapSuccesfully(0, 3, numShufflePartitions = 2) - completeNextShuffleMapSuccesfully(1, 3, numShufflePartitions = 1) + completeShuffleMapStageSuccessfully(0, 3, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(1, 3, numShufflePartitions = 1) // Fail stage 2 completeNextStageWithFetchFailure(2, 0, shuffleDepTwo) scheduler.resubmitFailedStages() // Rerun stage 0 - completeNextShuffleMapSuccesfully(0, 4, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(0, 4, numShufflePartitions = 2) // Now again, fail stage 1 (up to MAX_FAILURES) but confirm that this doesn't trigger an abort // since we succeeded in between @@ -738,8 +735,8 @@ class DAGSchedulerSuite // Next, succeed all and confirm output // Rerun stage 0 + 1 - completeNextShuffleMapSuccesfully(0, 5, numShufflePartitions = 2) - completeNextShuffleMapSuccesfully(1, 5, numShufflePartitions = 1) + completeShuffleMapStageSuccessfully(0, 5, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(1, 5, numShufflePartitions = 1) // Succeed stage 2 and verify results completeNextResultStageWithSuccess(2, 1) From 5bb1ae6fe5b7b44a432e35ea588a7d88fe2168eb Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 2 Sep 2015 16:36:08 -0700 Subject: [PATCH 45/45] Added more comments to clarify tricky test case --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 5cb7b81b29d68..fc0d25dedf03c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -697,6 +697,7 @@ class DAGSchedulerSuite val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) + // First, execute stages 0 and 1, failing stage 1 up to MAX-1 times. for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES - 1) { // Make each task in stage 0 success completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) @@ -712,19 +713,20 @@ class DAGSchedulerSuite assert(!ended) } - // Rerun stage 0 and 1 + // Rerun stage 0 and 1 to step through the task set completeShuffleMapStageSuccessfully(0, 3, numShufflePartitions = 2) completeShuffleMapStageSuccessfully(1, 3, numShufflePartitions = 1) - // Fail stage 2 + // Fail stage 2 so that stage 1 is resubmitted when we call scheduler.resubmitFailedStages() completeNextStageWithFetchFailure(2, 0, shuffleDepTwo) scheduler.resubmitFailedStages() - // Rerun stage 0 + + // Rerun stage 0 to step through the task set completeShuffleMapStageSuccessfully(0, 4, numShufflePartitions = 2) // Now again, fail stage 1 (up to MAX_FAILURES) but confirm that this doesn't trigger an abort - // since we succeeded in between + // since we succeeded in between. completeNextStageWithFetchFailure(1, 4, shuffleDepOne) scheduler.resubmitFailedStages()