-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-53575][CORE] Retry entire consumer stages when checksum mismatch detected for a retried shuffle map task #52336
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
2ab6a38
8644232
6e214f0
cd20fa4
e75e88a
bec3e9f
491a04b
0bd01f6
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 | ||||
|---|---|---|---|---|---|---|
|
|
@@ -1551,29 +1551,46 @@ private[spark] class DAGScheduler( | |||||
| // The operation here can make sure for the partially completed intermediate stage, | ||||||
| // `findMissingPartitions()` returns all partitions every time. | ||||||
| stage match { | ||||||
| case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => | ||||||
| // already executed at least once | ||||||
| if (sms.getNextAttemptId > 0) { | ||||||
| // While we previously validated possible rollbacks during the handling of a FetchFailure, | ||||||
| // where we were fetching from an indeterminate source map stages, this later check | ||||||
| // covers additional cases like recalculating an indeterminate stage after an executor | ||||||
| // loss. Moreover, because this check occurs later in the process, if a result stage task | ||||||
| // has successfully completed, we can detect this and abort the job, as rolling back a | ||||||
| // result stage is not possible. | ||||||
| val stagesToRollback = collectSucceedingStages(sms) | ||||||
| abortStageWithInvalidRollBack(stagesToRollback) | ||||||
| // stages which cannot be rolled back were aborted which leads to removing the | ||||||
| // the dependant job(s) from the active jobs set | ||||||
| val numActiveJobsWithStageAfterRollback = | ||||||
| activeJobs.count(job => stagesToRollback.contains(job.finalStage)) | ||||||
| if (numActiveJobsWithStageAfterRollback == 0) { | ||||||
| logInfo(log"All jobs depending on the indeterminate stage " + | ||||||
| log"(${MDC(STAGE_ID, stage.id)}) were aborted so this stage is not needed anymore.") | ||||||
| return | ||||||
| case sms: ShuffleMapStage if !sms.isAvailable => | ||||||
| val needFullStageRetry = if (sms.shuffleDep.checksumMismatchFullRetryEnabled) { | ||||||
|
Contributor
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. Catching up on PR's I missed out on reviewing. This negatively interacts if there is push based shuffle enabled. +CC @ivoson
Contributor
Author
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. Hi @mridulm can you please explain more about the issue with push based shuffle? Thanks.
Contributor
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. With push based shuffle enabled - a mappers output would also be pushed to mergers to create a reducer oriented view (all mappers write to a single merger for a given reducer). Essentially, for indeterminate stages, the entire reducer oriented view is unusable - and needs to be recomputed.
Contributor
Author
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. Hi @mridulm to recompute the indeterminate stages, we'll clean up all the shuffle outputs and shuffle merge state for push-based shuffle. Would that resolve your concern regarding to push-based shuffle? spark/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala Lines 1584 to 1585 in d0fbb15
|
||||||
| // When the parents of this stage are indeterminate (e.g., some parents are not | ||||||
| // checkpointed and checksum mismatches are detected), the output data of the parents | ||||||
| // may have changed due to task retries. For correctness reason, we need to | ||||||
| // retry all tasks of the current stage. The legacy way of using current stage's | ||||||
| // deterministic level to trigger full stage retry is not accurate. | ||||||
| stage.isParentIndeterminate | ||||||
| } else { | ||||||
| if (stage.isIndeterminate) { | ||||||
| // already executed at least once | ||||||
| if (sms.getNextAttemptId > 0) { | ||||||
| // While we previously validated possible rollbacks during the handling of a FetchFailure, | ||||||
| // where we were fetching from an indeterminate source map stages, this later check | ||||||
| // covers additional cases like recalculating an indeterminate stage after an executor | ||||||
| // loss. Moreover, because this check occurs later in the process, if a result stage task | ||||||
| // has successfully completed, we can detect this and abort the job, as rolling back a | ||||||
| // result stage is not possible. | ||||||
| val stagesToRollback = collectSucceedingStages(sms) | ||||||
| abortStageWithInvalidRollBack(stagesToRollback) | ||||||
|
Comment on lines
+1572
to
+1573
Contributor
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. nit: We could have delegated this to
Contributor
Author
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. |
||||||
| // stages which cannot be rolled back were aborted which leads to removing the | ||||||
| // the dependant job(s) from the active jobs set | ||||||
| val numActiveJobsWithStageAfterRollback = | ||||||
| activeJobs.count(job => stagesToRollback.contains(job.finalStage)) | ||||||
| if (numActiveJobsWithStageAfterRollback == 0) { | ||||||
| logInfo(log"All jobs depending on the indeterminate stage " + | ||||||
| log"(${MDC(STAGE_ID, stage.id)}) were aborted so this stage is not needed anymore.") | ||||||
| return | ||||||
| } | ||||||
| } | ||||||
| true | ||||||
| } else { | ||||||
| false | ||||||
| } | ||||||
| } | ||||||
| mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) | ||||||
| sms.shuffleDep.newShuffleMergeState() | ||||||
|
|
||||||
| if (needFullStageRetry) { | ||||||
| mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) | ||||||
| sms.shuffleDep.newShuffleMergeState() | ||||||
| } | ||||||
| case _ => | ||||||
| } | ||||||
|
|
||||||
|
|
@@ -1886,6 +1903,20 @@ private[spark] class DAGScheduler( | |||||
| } | ||||||
| } | ||||||
|
|
||||||
| /** | ||||||
| * If a map stage is non-deterministic, the map tasks of the stage may return different result | ||||||
| * when re-try. To make sure data correctness, we need to re-try all the tasks of its succeeding | ||||||
| * stages, as the input data may be changed after the map tasks are re-tried. For stages where | ||||||
| * rollback and retry all tasks are not possible, we will need to abort the stages. | ||||||
| */ | ||||||
| private[scheduler] def abortUnrollbackableStages(mapStage: ShuffleMapStage): Unit = { | ||||||
| val stagesToRollback = collectSucceedingStages(mapStage) | ||||||
| val rollingBackStages = abortStageWithInvalidRollBack(stagesToRollback) | ||||||
| logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output " + | ||||||
| log"was failed, we will roll back and rerun below stages which include itself and all its " + | ||||||
| log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") | ||||||
| } | ||||||
|
|
||||||
| /** | ||||||
| * 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. | ||||||
|
|
@@ -2022,8 +2053,26 @@ private[spark] class DAGScheduler( | |||||
| // The epoch of the task is acceptable (i.e., the task was launched after the most | ||||||
| // recent failure we're aware of for the executor), so mark the task's output as | ||||||
| // available. | ||||||
| mapOutputTracker.registerMapOutput( | ||||||
| val isChecksumMismatched = mapOutputTracker.registerMapOutput( | ||||||
| shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) | ||||||
| if (isChecksumMismatched) { | ||||||
| shuffleStage.isChecksumMismatched = isChecksumMismatched | ||||||
|
Contributor
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 never reset back to Not sure what I am missing here.
Contributor
Author
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. Hi @mridulm , this is not set back to This won't fail the app, the impact is that the succeeding stages would have a fully-retry. The code logic has changed a little bit in PR: #53274 Pls take a look once you get a change. Thanks.
Contributor
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. On retry - when we throw away the entire mapper output and recompute it -> at which point, we can go back to setting it to
Contributor
Author
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. Currently, it's not setting back to |
||||||
| // There could be multiple checksum mismatches detected for a single stage attempt. | ||||||
| // We check for stage abortion once and only once when we first detect checksum | ||||||
| // mismatch for each stage attempt. For example, assume that we have | ||||||
| // stage1 -> stage2, and we encounter checksum mismatch during the retry of stage1. | ||||||
| // In this case, we need to call abortUnrollbackableStages() for the succeeding | ||||||
| // stages. Assume that when stage2 is retried, some tasks finish and some tasks | ||||||
| // failed again with FetchFailed. In case that we encounter checksum mismatch again | ||||||
| // during the retry of stage1, we need to call abortUnrollbackableStages() again. | ||||||
| if (shuffleStage.maxChecksumMismatchedId < smt.stageAttemptId) { | ||||||
| shuffleStage.maxChecksumMismatchedId = smt.stageAttemptId | ||||||
| if (shuffleStage.shuffleDep.checksumMismatchFullRetryEnabled | ||||||
| && shuffleStage.isStageIndeterminate) { | ||||||
| abortUnrollbackableStages(shuffleStage) | ||||||
| } | ||||||
| } | ||||||
| } | ||||||
| } | ||||||
| } else { | ||||||
| logInfo(log"Ignoring ${MDC(TASK_NAME, smt)} completion from an older attempt of indeterminate stage") | ||||||
|
|
@@ -2148,12 +2197,8 @@ private[spark] class DAGScheduler( | |||||
| // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is | ||||||
| // guaranteed to be determinate, so the input data of the reducers will not change | ||||||
| // even if the map tasks are re-tried. | ||||||
| if (mapStage.isIndeterminate) { | ||||||
| val stagesToRollback = collectSucceedingStages(mapStage) | ||||||
| val rollingBackStages = abortStageWithInvalidRollBack(stagesToRollback) | ||||||
| logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + | ||||||
| log"we will roll back and rerun below stages which include itself and all its " + | ||||||
| log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") | ||||||
| if (mapStage.isIndeterminate && !mapStage.shuffleDep.checksumMismatchFullRetryEnabled) { | ||||||
| abortUnrollbackableStages(mapStage) | ||||||
| } | ||||||
|
|
||||||
| // We expect one executor failure to trigger many FetchFailures in rapid succession, | ||||||
|
|
||||||
Uh oh!
There was an error while loading. Please reload this page.