-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-7308] prevent concurrent attempts for one stage #5964
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
d08c20c
89e8428
70a787b
7fbcefb
2eebbf2
7142242
ccaa159
3585b96
de23530
c91ee10
05c72fd
5dc5436
37eece8
31c21fa
a894be1
93592b1
ea2d972
de0a596
6654c53
dd2839d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -830,6 +830,10 @@ class DAGScheduler( | |
| logDebug("submitMissingTasks(" + stage + ")") | ||
| // Get our pending tasks and remember them in our pendingTasks entry | ||
| stage.pendingTasks.clear() | ||
| stage match { | ||
| case smt: ShuffleMapStage => smt.clearPartitionComputeCount() | ||
| case _ => | ||
| } | ||
|
|
||
|
|
||
| // First figure out the indexes of partition ids to compute. | ||
|
|
@@ -891,7 +895,7 @@ class DAGScheduler( | |
| partitionsToCompute.map { id => | ||
| val locs = getPreferredLocs(stage.rdd, id) | ||
| val part = stage.rdd.partitions(id) | ||
| new ShuffleMapTask(stage.id, taskBinary, part, locs) | ||
| new ShuffleMapTask(stage.id, stage.attemptId, taskBinary, part, locs) | ||
| } | ||
|
|
||
| case stage: ResultStage => | ||
|
|
@@ -900,7 +904,7 @@ class DAGScheduler( | |
| val p: Int = job.partitions(id) | ||
| val part = stage.rdd.partitions(p) | ||
| val locs = getPreferredLocs(stage.rdd, p) | ||
| new ResultTask(stage.id, taskBinary, part, locs, id) | ||
| new ResultTask(stage.id, stage.attemptId, taskBinary, part, locs, id) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -974,6 +978,7 @@ class DAGScheduler( | |
| val stageId = task.stageId | ||
| val taskType = Utils.getFormattedClassName(task) | ||
|
|
||
| // REVIEWERS: does this need special handling for multiple completions of the same task? | ||
| outputCommitCoordinator.taskCompleted(stageId, task.partitionId, | ||
| event.taskInfo.attempt, event.reason) | ||
|
|
||
|
|
@@ -1031,15 +1036,31 @@ class DAGScheduler( | |
|
|
||
| case smt: ShuffleMapTask => | ||
| val shuffleStage = stage.asInstanceOf[ShuffleMapStage] | ||
| val computeCount = shuffleStage.incComputeCount(smt.partitionId) | ||
| updateAccumulators(event) | ||
| val status = event.result.asInstanceOf[MapStatus] | ||
| val execId = status.location.executorId | ||
| logDebug("ShuffleMapTask finished on " + execId) | ||
| if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { | ||
| logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) | ||
| if (computeCount > 1) { | ||
| // REVIEWERS: do I need to worry about speculation here, when multiple completion | ||
| // events are normal? | ||
|
|
||
| // REVIEWERS: is this really only a problem on a ShuffleMapTask?? does it also cause | ||
| // problems for ResultTask? | ||
|
|
||
| // This can happen when a retry runs a task, but there was a lingering task from an | ||
| // earlier attempt which also finished. The results might be OK, or they might not. | ||
| // To be safe, we'll retry the task, and do it in yet another attempt, to avoid more | ||
| // task output clobbering. | ||
| logInfo(s"Multiple completion events for task $task. Results may be corrupt," + | ||
| s" assuming task needs to be rerun.") | ||
| shuffleStage.removeOutputLoc(task.partitionId) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. for (3) & (4) |
||
| } else if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { | ||
| logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") | ||
| } else { | ||
| shuffleStage.addOutputLoc(smt.partitionId, status) | ||
| } | ||
|
|
||
| if (runningStages.contains(shuffleStage) && shuffleStage.pendingTasks.isEmpty) { | ||
| markStageAsFinished(shuffleStage) | ||
| logInfo("looking for newly runnable stages") | ||
|
|
@@ -1103,9 +1124,14 @@ class DAGScheduler( | |
| // multiple tasks running concurrently on different executors). In that case, it is possible | ||
| // the fetch failure has already been handled by the scheduler. | ||
| if (runningStages.contains(failedStage)) { | ||
| logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + | ||
| s"due to a fetch failure from $mapStage (${mapStage.name})") | ||
| markStageAsFinished(failedStage, Some(failureMessage)) | ||
| if (failedStage.attemptId - 1 > task.stageAttemptId) { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This addresses issue (1)-- if we get a fetch failure, but we've already failed the attempt of the stage that caused the fetch failure, then do not resubmit the stage again. (Lots of other small changes to add stageAttemptId to the task) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This block is a little awkward. Being a little more explicit would be good here. Something like this: //it is possible failure has already been handled by the scheduler.
val failureRequiresHandling = runningStages.contains(failedStage);
if (failureRequireHandling) {
val stageHasFailed = failedStage.attemptId - 1 > task.stageAttemptId;
if (stageHasFailed) {
...
}
}There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm confused about how this works. Doesn't the stage still get added to failed stages on line 1149, so it will still be resubmitted? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hmm, good point. I think it works in my existing test case, because (btw, thanks for looking at it in this state, I do still plan on splitting this apart some, just keep getting sidetracked ...) |
||
| logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + | ||
| s" ${task.stageAttemptId}, which has already failed") | ||
| } else { | ||
| logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + | ||
| s"due to a fetch failure from $mapStage (${mapStage.name})") | ||
| markStageAsFinished(failedStage, Some(failureMessage)) | ||
| } | ||
| } | ||
|
|
||
| if (disallowStageRetryForTest) { | ||
|
|
@@ -1128,6 +1154,16 @@ class DAGScheduler( | |
| mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) | ||
| } | ||
|
|
||
| // We also have to mark this map output as unavailable. Its possible that a *later* attempt | ||
| // has finished this task in the meantime, but when this task fails, it might end up | ||
| // deleting the mapOutput from the earlier successful attempt. | ||
| failedStage match { | ||
| case smt: ShuffleMapStage => | ||
| smt.incComputeCount(reduceId) | ||
| smt.removeOutputLoc(reduceId) | ||
| case _ => | ||
| } | ||
|
|
||
| // TODO: mark the executor as failed only if there were lots of fetch failures on it | ||
| if (bmAddress != null) { | ||
| handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -159,6 +159,12 @@ private[spark] class TaskSchedulerImpl( | |
| this.synchronized { | ||
| val manager = createTaskSetManager(taskSet, maxTaskFailures) | ||
| activeTaskSets(taskSet.id) = manager | ||
| val taskSetsPerStage = activeTaskSets.values.filterNot(_.isZombie).groupBy(_.stageId) | ||
| taskSetsPerStage.foreach { case (stage, taskSets) => | ||
| if (taskSets.size > 1) { | ||
| throw new SparkIllegalStateException("more than one active taskSet for stage " + stage) | ||
| } | ||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is defensive. Hopefully this PR will eliminate multiple concurrent attempts for the same stage, but I'd like to add this check in any case. |
||
| schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) | ||
|
|
||
| if (!isLocal && !hasReceivedTask) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -74,12 +74,16 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB | |
| * end of the output file. This will be used by getBlockLocation to figure out where each block | ||
| * begins and ends. | ||
| * */ | ||
| def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { | ||
| def writeIndexFile( | ||
| shuffleId: Int, | ||
| mapId: Int, | ||
| lengths: Array[Long], | ||
| initialFileLength: Long): Unit = { | ||
| val indexFile = getIndexFile(shuffleId, mapId) | ||
| val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) | ||
| Utils.tryWithSafeFinally { | ||
| // We take in lengths of each block, need to convert it to offsets. | ||
| var offset = 0L | ||
| var offset = initialFileLength | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. to fix issue (2) -- data files are appended, but before this, index files always pointed to the beginning of the data file. |
||
| out.writeLong(offset) | ||
| for (length <- lengths) { | ||
| offset += length | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this (and everything related to
partitionComputeCount) is for issues (3) & (4)