Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
112 commits
Select commit Hold shift + click to select a range
e6e483c
[SPARK-9679] [ML] [PYSPARK] Add Python API for Stop Words Remover
holdenk Sep 1, 2015
3f63bd6
[SPARK-10398] [DOCS] Migrate Spark download page to use new lua mirro…
srowen Sep 1, 2015
ec01280
[SPARK-4223] [CORE] Support * in acls.
Sep 1, 2015
bf550a4
[SPARK-10162] [SQL] Fix the timezone omitting for PySpark Dataframe f…
0x0FFF Sep 1, 2015
00d9af5
[SPARK-10392] [SQL] Pyspark - Wrong DateType support on JDBC connection
0x0FFF Sep 1, 2015
c3b881a
[SPARK-7336] [HISTORYSERVER] Fix bug that applications status incorre…
ArcherShao Sep 2, 2015
56c4c17
[SPARK-10034] [SQL] add regression test for Sort on Aggregate
cloud-fan Sep 2, 2015
fc48307
[SPARK-10389] [SQL] support order by non-attribute grouping expressio…
cloud-fan Sep 2, 2015
2da3a9e
[SPARK-10004] [SHUFFLE] Perform auth checks when clients read shuffle…
Sep 2, 2015
6cd98c1
[SPARK-10417] [SQL] Iterating through Column results in infinite loop
0x0FFF Sep 2, 2015
03f3e91
[SPARK-10422] [SQL] String column in InMemoryColumnarCache needs to o…
yhuai Sep 3, 2015
44948a2
[SPARK-9723] [ML] params getordefault should throw more useful error
holdenk Sep 3, 2015
4bd85d0
[SPARK-5945] Spark should not retry a stage infinitely on a FetchFail…
Sep 3, 2015
0985d2c
[SPARK-8707] RDD#toDebugString fails if any cached RDD has invalid pa…
navis Sep 3, 2015
f6c447f
Removed code duplication in ShuffleBlockFetcherIterator
eracah Sep 3, 2015
3ddb9b3
[SPARK-10247] [CORE] improve readability of a test case in DAGSchedul…
squito Sep 3, 2015
62b4690
[SPARK-10379] preserve first page in UnsafeShuffleExternalSorter
Sep 3, 2015
0349b5b
[SPARK-10411] [SQL] Move visualization above explain output and hide …
zsxwing Sep 3, 2015
67580f1
[SPARK-10332] [CORE] Fix yarn spark executor validation
holdenk Sep 3, 2015
3abc0d5
[SPARK-9596] [SQL] treat hadoop classes as shared one in IsolatedClie…
WangTaoTheTonic Sep 3, 2015
af0e312
[SPARK-8951] [SPARKR] support Unicode characters in collect()
Sep 3, 2015
49aff7b
[SPARK-10432] spark.port.maxRetries documentation is unclear
Sep 3, 2015
d911c68
[SPARK-10431] [CORE] Fix intermittent test failure. Wait for event qu…
Sep 3, 2015
754f853
[SPARK-9869] [STREAMING] Wait for all event notifications before asse…
Sep 3, 2015
e62f4a4
[SPARK-9672] [MESOS] Don’t include SPARK_ENV_LOADED when passing env …
pashields Sep 3, 2015
11ef32c
[SPARK-10430] [CORE] Added hashCode methods in AccumulableInfo and RD…
Sep 3, 2015
db4c130
[SPARK-9591] [CORE] Job may fail for exception during getting remote …
jeanlyn Sep 3, 2015
08b0750
[SPARK-10435] Spark submit should fail fast for Mesos cluster mode wi…
Sep 3, 2015
208fbca
[SPARK-10421] [BUILD] Exclude curator artifacts from tachyon dependen…
Sep 3, 2015
cf42138
[SPARK-10003] Improve readability of DAGScheduler
Sep 4, 2015
143e521
[MINOR] Minor style fix in SparkR
shivaram Sep 4, 2015
804a012
MAINTENANCE: Automated closing of pull requests.
marmbrus Sep 4, 2015
c3c0e43
[SPARK-10176] [SQL] Show partially analyzed plans when checkAnswer fa…
cloud-fan Sep 4, 2015
3339e6f
[SPARK-10450] [SQL] Minor improvements to readability / style / typos…
Sep 4, 2015
b087d23
[SPARK-9669] [MESOS] Support PySpark on Mesos cluster mode.
tnachen Sep 4, 2015
2e1c175
[SPARK-10454] [SPARK CORE] wait for empty event queue
Sep 4, 2015
eafe372
[SPARK-10311] [STREAMING] Reload appId and attemptId when app starts …
XuTingjun Sep 4, 2015
22eab70
[SPARK-10402] [DOCS] [ML] Add defaults to the scaladoc for params in ml/
holdenk Sep 5, 2015
47058ca
[SPARK-9925] [SQL] [TESTS] Set SQLConf.SHUFFLE_PARTITIONS.key correct…
yhuai Sep 5, 2015
6c75194
[HOTFIX] [SQL] Fixes compilation error
liancheng Sep 5, 2015
7a4f326
[SPARK-10440] [STREAMING] [DOCS] Update python API stuff in the progr…
tdas Sep 5, 2015
bca8c07
[SPARK-10434] [SQL] Fixes Parquet schema of arrays that may contain null
liancheng Sep 5, 2015
871764c
[SPARK-10013] [ML] [JAVA] [TEST] remove java assert from java unit tests
holdenk Sep 5, 2015
5ffe752
[SPARK-9767] Remove ConnectionManager.
rxin Sep 7, 2015
9d8e838
[DOC] Added R to the list of languages with "high-level API" support …
Sep 8, 2015
6ceed85
Docs small fixes
jaceklaskowski Sep 8, 2015
990c9f7
[SPARK-9170] [SQL] Use OrcStructInspector to be case preserving when …
viirya Sep 8, 2015
5b2192e
[SPARK-10480] [ML] Fix ML.LinearRegressionModel.copy()
yanboliang Sep 8, 2015
5fd5795
[SPARK-10316] [SQL] respect nondeterministic expressions in PhysicalO…
cloud-fan Sep 8, 2015
f7b55db
[SPARK-10470] [ML] ml.IsotonicRegressionModel.copy should set parent
yanboliang Sep 8, 2015
7a9dcbc
[SPARK-10441] [SQL] Save data correctly to json.
yhuai Sep 8, 2015
e6f8d36
[SPARK-10468] [ MLLIB ] Verify schema before Dataframe select API call
Sep 8, 2015
52b24a6
[SPARK-10492] [STREAMING] [DOCUMENTATION] Update Streaming documentat…
tdas Sep 8, 2015
d637a66
[SPARK-10327] [SQL] Cache Table is not working while subquery has ali…
chenghao-intel Sep 8, 2015
2143d59
[HOTFIX] Fix build break caused by #8494
marmbrus Sep 8, 2015
ae74c3f
[RELEASE] Add more contributors & only show names in release notes.
rxin Sep 9, 2015
820913f
[SPARK-10071] [STREAMING] Output a warning when writing QueueInputDSt…
zsxwing Sep 9, 2015
52fe32f
[SPARK-9834] [MLLIB] implement weighted least squares via normal equa…
mengxr Sep 9, 2015
a157348
[SPARK-10464] [MLLIB] Add WeibullGenerator for RandomDataGenerator
yanboliang Sep 9, 2015
3a11e50
[SPARK-10373] [PYSPARK] move @since into pyspark from sql
Sep 9, 2015
0e2f216
[SPARK-10094] Pyspark ML Feature transformers marked as experimental
noel-smith Sep 9, 2015
2f6fd52
[SPARK-9654] [ML] [PYSPARK] Add IndexToString to PySpark
holdenk Sep 9, 2015
91a577d
[SPARK-10249] [ML] [DOC] Add Python Code Example to StopWordsRemover …
hhbyyh Sep 9, 2015
c1bc4f4
[SPARK-10227] fatal warnings with sbt on Scala 2.11
Sep 9, 2015
2ddeb63
[SPARK-10117] [MLLIB] Implement SQL data source API for reading LIBSV…
Lewuathe Sep 9, 2015
c0052d8
[SPARK-10481] [YARN] SPARK_PREPEND_CLASSES make spark-yarn related ja…
zjffdu Sep 9, 2015
71da163
[SPARK-10461] [SQL] make sure `input.primitive` is always variable na…
cloud-fan Sep 9, 2015
45de518
[SPARK-9730] [SQL] Add Full Outer Join support for SortMergeJoin
viirya Sep 9, 2015
56a0fe5
[SPARK-9772] [PYSPARK] [ML] Add Python API for ml.feature.VectorSlicer
yanboliang Sep 10, 2015
1dc7548
[MINOR] [MLLIB] [ML] [DOC] fixed typo: label for negative result shou…
sparadiso Sep 10, 2015
48817cc
[SPARK-10497] [BUILD] [TRIVIAL] Handle both locations for JIRAError w…
holdenk Sep 10, 2015
4f1daa1
[SPARK-10065] [SQL] avoid the extra copy when generate unsafe array
cloud-fan Sep 10, 2015
f892d92
[SPARK-7142] [SQL] Minor enhancement to BooleanSimplification Optimiz…
Sep 10, 2015
49da38e
[SPARK-10301] [SPARK-10428] [SQL] Addresses comments of PR #8583 and …
liancheng Sep 10, 2015
e048111
[SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill
chenghao-intel Sep 10, 2015
a76bde9
[SPARK-10469] [DOC] Try and document the three options
holdenk Sep 10, 2015
af3bc59
[SPARK-8167] Make tasks that fail from YARN preemption not fail job
mccheah Sep 10, 2015
f0562e8
[SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecut…
dragos Sep 10, 2015
a5ef2d0
[SPARK-10514] [MESOS] waiting for min no of total cores acquired by S…
SleepyThread Sep 10, 2015
d88abb7
[SPARK-9990] [SQL] Create local hash join operator
zsxwing Sep 10, 2015
45e3be5
[SPARK-10049] [SPARKR] Support collecting data of ArraryType in DataF…
Sep 10, 2015
3db7255
[SPARK-10443] [SQL] Refactor SortMergeOuterJoin to reduce duplication
Sep 10, 2015
4204757
Add 1.5 to master branch EC2 scripts
shivaram Sep 10, 2015
89562a1
[SPARK-7544] [SQL] [PySpark] pyspark.sql.types.Row implements __getit…
yanboliang Sep 10, 2015
0eabea8
[SPARK-9043] Serialize key, value and combiner classes in ShuffleDepe…
massie Sep 11, 2015
339a527
[SPARK-10023] [ML] [PySpark] Unified DecisionTreeParams checkpointInt…
yanboliang Sep 11, 2015
a140dd7
[SPARK-10027] [ML] [PySpark] Add Python API missing methods for ml.fe…
yanboliang Sep 11, 2015
e1d7f64
[SPARK-10472] [SQL] Fixes DataType.typeName for UDT
liancheng Sep 11, 2015
9bbe33f
[SPARK-10556] Remove explicit Scala version for sbt project build files
ahirreddy Sep 11, 2015
c268ca4
[SPARK-10518] [DOCS] Update code examples in spark.ml user guide to u…
y-shimizu Sep 11, 2015
b656e61
[SPARK-10026] [ML] [PySpark] Implement some common Params for regress…
yanboliang Sep 11, 2015
b01b262
[SPARK-9773] [ML] [PySpark] Add Python API for MultilayerPerceptronCl…
yanboliang Sep 11, 2015
960d2d0
[SPARK-10537] [ML] document LIBSVM source options in public API doc a…
mengxr Sep 11, 2015
2e3a280
[MINOR] [MLLIB] [ML] [DOC] Minor doc fixes for StringIndexer and Meta…
jkbradley Sep 11, 2015
6ce0886
[SPARK-10540] [SQL] Ignore HadoopFsRelationTest's "test all data type…
yhuai Sep 11, 2015
5f46444
[SPARK-8530] [ML] add python API for MinMaxScaler
hhbyyh Sep 11, 2015
b231ab8
[SPARK-10546] Check partitionId's range in ExternalSorter#spill()
tedyu Sep 11, 2015
c373866
[PYTHON] Fixed typo in exception message
icaromedeiros Sep 11, 2015
d5d6473
[SPARK-10442] [SQL] fix string to boolean cast
cloud-fan Sep 11, 2015
1eede3b
[SPARK-7142] [SQL] Minor enhancement to BooleanSimplification Optimiz…
Sep 11, 2015
e626ac5
[SPARK-9992] [SPARK-9994] [SPARK-9998] [SQL] Implement the local TopK…
zsxwing Sep 11, 2015
c2af42b
[SPARK-9990] [SQL] Local hash join follow-ups
Sep 11, 2015
d74c6a1
[SPARK-10564] ThreadingSuite: assertion failures in threads don't fai…
Sep 11, 2015
c34fc19
[SPARK-9014] [SQL] Allow Python spark API to use built-in exponential…
0x0FFF Sep 11, 2015
6d83678
[SPARK-10566] [CORE] SnappyCompressionCodec init exception handling m…
dimfeld Sep 12, 2015
8285e3b
[SPARK-10554] [CORE] Fix NPE with ShutdownHook
Sep 12, 2015
22730ad
[SPARK-10547] [TEST] Streamline / improve style of Java API tests
srowen Sep 12, 2015
f4a2280
[SPARK-6548] Adding stddev to DataFrame functions
JihongMA Sep 12, 2015
b3a7480
[SPARK-10330] Add Scalastyle rule to require use of SparkHadoopUtil J…
JoshRosen Sep 12, 2015
1dc614b
[SPARK-10222] [GRAPHX] [DOCS] More thoroughly deprecate Bagel in favo…
srowen Sep 13, 2015
7d94924
Deprecates SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC
liancheng Sep 1, 2015
85bbfde
Removes instead of deprecates the old option
liancheng Sep 2, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-5945] Spark should not retry a stage infinitely on a FetchFail…
…edException

The ```Stage``` class now tracks whether there were a sufficient number of consecutive failures of that stage to trigger an abort.

To avoid an infinite loop of stage retries, we abort the job completely after 4 consecutive stage failures for one stage. We still allow more than 4 consecutive stage failures if there is an intervening successful attempt for the stage, so that in very long-lived applications, where a stage may get reused many times, we don't abort the job after failures that have been recovered from successfully.

I've added test cases to exercise the most obvious scenarios.

Author: Ilya Ganelin <[email protected]>

Closes apache#5636 from ilganeli/SPARK-5945.
  • Loading branch information
Ilya Ganelin authored and Andrew Or committed Sep 3, 2015
commit 4bd85d06e0334c49be18c4612b04d013b37f189c
Original file line number Diff line number Diff line change
Expand Up @@ -1101,7 +1101,6 @@ class DAGScheduler(
s" ${task.stageAttemptId} and there is a more recent attempt for that stage " +
s"(attempt ID ${failedStage.latestInfo.attemptId}) running")
} else {

// It is likely that we receive multiple FetchFailed for a single stage (because we have
// multiple tasks running concurrently on different executors). In that case, it is
// possible the fetch failure has already been handled by the scheduler.
Expand All @@ -1117,6 +1116,11 @@ class DAGScheduler(
if (disallowStageRetryForTest) {
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 " +
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
// in that case the event will already have been scheduled.
Expand Down Expand Up @@ -1240,10 +1244,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,so that if a stage is
// 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)
logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime))
}

outputCommitCoordinator.stageEnd(stage.id)
listenerBus.post(SparkListenerStageCompleted(stage.latestInfo))
runningStages -= stage
Expand Down
30 changes: 29 additions & 1 deletion core/src/main/scala/org/apache/spark/scheduler/Stage.scala
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ import org.apache.spark.util.CallSite
* be updated for each attempt.
*
*/
private[spark] abstract class Stage(
private[scheduler] abstract class Stage(
val id: Int,
val rdd: RDD[_],
val numTasks: Int,
Expand Down Expand Up @@ -92,6 +92,29 @@ private[spark] abstract class Stage(
*/
private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId)

/**
* 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 (SPARK-5945).
*/
private val fetchFailedAttemptIds = new HashSet[Int]

private[scheduler] def clearFailures() : Unit = {
fetchFailedAttemptIds.clear()
}

/**
* 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 failedOnFetchAndShouldAbort(stageAttemptId: Int): Boolean = {
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. */
def makeNewStageAttempt(
numPartitionsToCompute: Int,
Expand All @@ -110,3 +133,8 @@ private[spark] abstract class Stage(
case _ => false
}
}

private[scheduler] object Stage {
// The number of consecutive failures allowed before a stage is aborted
val MAX_CONSECUTIVE_FETCH_FAILURES = 4
}
282 changes: 279 additions & 3 deletions core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down Expand Up @@ -473,6 +473,282 @@ class DAGSchedulerSuite
assertDataStructuresEmpty()
}


// Helper function to validate state when creating tests for task failures
private def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) {
assert(stageAttempt.stageId === stageId)
assert(stageAttempt.stageAttemptId == attempt)
}


// Helper functions to extract commonly used code in Fetch Failure test cases
private def setupStageAbortTest(sc: SparkContext) {
sc.listenerBus.addListener(new EndListener())
ended = false
jobResult = null
}

// Create a new Listener to confirm that the listenerBus sees the JobEnd message
// when we abort the stage. This message will also be consumed by the EventLoggingListener
// so this will propagate up to the user.
var ended = false
var jobResult : JobResult = null

class EndListener extends SparkListener {
override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
jobResult = jobEnd.jobResult
ended = true
}
}

/**
* Common code to get the next stage attempt, confirm it's the one we expect, and complete it
* successfully.
*
* @param stageId - The current stageId
* @param attemptIdx - The current attempt count
* @param numShufflePartitions - The number of partitions in the next stage
*/
private def completeShuffleMapStageSuccessfully(
stageId: Int,
attemptIdx: Int,
numShufflePartitions: Int): Unit = {
val stageAttempt = taskSets.last
checkStageId(stageId, attemptIdx, stageAttempt)
complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map {
case (task, idx) =>
(Success, makeMapStatus("host" + ('A' + idx).toChar, numShufflePartitions))
}.toSeq)
}

/**
* Common code to get the next stage attempt, confirm it's the one we expect, and complete it
* with all FetchFailure.
*
* @param stageId - The current stageId
* @param attemptIdx - The current attempt count
* @param shuffleDep - The shuffle dependency of the stage with a fetch failure
*/
private def completeNextStageWithFetchFailure(
stageId: Int,
attemptIdx: Int,
shuffleDep: ShuffleDependency[_, _, _]): Unit = {
val stageAttempt = taskSets.last
checkStageId(stageId, attemptIdx, stageAttempt)
complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) =>
(FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null)
}.toSeq)
}

/**
* Common code to get the next result stage attempt, confirm it's the one we expect, and
* complete it with a success where we return 42.
*
* @param stageId - The current stageId
* @param attemptIdx - The current attempt count
*/
private def completeNextResultStageWithSuccess(stageId: Int, attemptIdx: Int): Unit = {
val stageAttempt = taskSets.last
checkStageId(stageId, attemptIdx, stageAttempt)
assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage])
complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq)
}

/**
* In this test, we simulate a job where many tasks in the same stage fail. We want to show
* that many fetch failures inside a single stage attempt do not trigger an abort
* on their own, but only when there are enough failing stage attempts.
*/
test("Single stage fetch failure should not abort the stage.") {
setupStageAbortTest(sc)

val parts = 8
val shuffleMapRdd = new MyRDD(sc, parts, Nil)
val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
val shuffleId = shuffleDep.shuffleId
val reduceRdd = new MyRDD(sc, parts, List(shuffleDep))
submit(reduceRdd, (0 until parts).toArray)

completeShuffleMapStageSuccessfully(0, 0, numShufflePartitions = parts)

completeNextStageWithFetchFailure(1, 0, shuffleDep)

// 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"
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)
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
* trigger an overall job abort to avoid endless retries.
*/
test("Multiple consecutive stage fetch failures should lead to job being aborted.") {
setupStageAbortTest(sc)

val shuffleMapRdd = new MyRDD(sc, 2, Nil)
val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
val shuffleId = shuffleDep.shuffleId
val reduceRdd = new MyRDD(sc, 2, List(shuffleDep))
submit(reduceRdd, Array(0, 1))

for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES) {
// Complete all the tasks for the current attempt of stage 0 successfully
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 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_FETCH_FAILURES - 1) {
assert(scheduler.runningStages.nonEmpty)
assert(!ended)
} else {
// Stage should have been aborted and removed from running stages
assertDataStructuresEmpty()
sc.listenerBus.waitUntilEmpty(1000)
assert(ended)
jobResult match {
case JobFailed(reason) =>
assert(reason.getMessage.contains("ResultStage 1 () has failed the maximum"))
case other => fail(s"expected JobFailed, not $other")
}
}
}
}

/**
* In this test, we 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)

val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache()
val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)).cache()
val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null)
val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo))
submit(finalRdd, Array(0))

// In the first two iterations, Stage 0 succeeds and stage 1 fails. In the next two iterations,
// stage 2 fails.
for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES) {
// Complete all the tasks for the current attempt of stage 0 successfully
completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 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 {
completeShuffleMapStageSuccessfully(1, attempt, numShufflePartitions = 1)

// Fail stage 2
completeNextStageWithFetchFailure(2, attempt - Stage.MAX_CONSECUTIVE_FETCH_FAILURES / 2,
shuffleDepTwo)
}

// 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()
}

completeShuffleMapStageSuccessfully(0, 4, numShufflePartitions = 2)
completeShuffleMapStageSuccessfully(1, 4, numShufflePartitions = 1)

// Succeed stage2 with a "42"
completeNextResultStageWithSuccess(2, Stage.MAX_CONSECUTIVE_FETCH_FAILURES/2)

assert(results === Map(0 -> 42))
assertDataStructuresEmpty()
}

/**
* In this test we demonstrate that only consecutive failures trigger a stage abort. A stage may
* 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("Non-consecutive stage failures don't trigger abort") {
setupStageAbortTest(sc)

val shuffleOneRdd = new MyRDD(sc, 2, Nil).cache()
val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)).cache()
val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null)
val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo))
submit(finalRdd, Array(0))

// 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)

// Now we should have a new taskSet, for a new attempt of stage 1.
// Fail these tasks with FetchFailure
completeNextStageWithFetchFailure(1, attempt, shuffleDepOne)

scheduler.resubmitFailedStages()

// Confirm we have not yet aborted
assert(scheduler.runningStages.nonEmpty)
assert(!ended)
}

// Rerun stage 0 and 1 to step through the task set
completeShuffleMapStageSuccessfully(0, 3, numShufflePartitions = 2)
completeShuffleMapStageSuccessfully(1, 3, numShufflePartitions = 1)

// Fail stage 2 so that stage 1 is resubmitted when we call scheduler.resubmitFailedStages()
completeNextStageWithFetchFailure(2, 0, shuffleDepTwo)

scheduler.resubmitFailedStages()

// 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.
completeNextStageWithFetchFailure(1, 4, shuffleDepOne)

scheduler.resubmitFailedStages()

// Confirm we have not yet aborted
assert(scheduler.runningStages.nonEmpty)
assert(!ended)

// Next, succeed all and confirm output
// Rerun stage 0 + 1
completeShuffleMapStageSuccessfully(0, 5, numShufflePartitions = 2)
completeShuffleMapStageSuccessfully(1, 5, numShufflePartitions = 1)

// Succeed stage 2 and verify results
completeNextResultStageWithSuccess(2, 1)

assertDataStructuresEmpty()
sc.listenerBus.waitUntilEmpty(1000)
assert(ended === true)
assert(results === Map(0 -> 42))
}

test("trivial shuffle with multiple fetch failures") {
val shuffleMapRdd = new MyRDD(sc, 2, Nil)
val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
Expand Down Expand Up @@ -810,15 +1086,15 @@ 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))))
// complete stage 1
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:
Expand Down