-
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 3 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 |
|---|---|---|
|
|
@@ -307,6 +307,9 @@ private[spark] class DAGScheduler( | |
|
|
||
| private val shuffleFinalizeRpcThreads = sc.conf.get(config.PUSH_SHUFFLE_FINALIZE_RPC_THREADS) | ||
|
|
||
| private val checksumMismatchFullRetryEnabled = | ||
| sc.getConf.get(config.SCHEDULER_CHECKSUM_MISMATCH_FULL_RETRY_ENABLED) | ||
|
|
||
| // Since SparkEnv gets initialized after DAGScheduler, externalShuffleClient needs to be | ||
| // initialized lazily | ||
| private lazy val externalShuffleClient: Option[BlockStoreClient] = | ||
|
|
@@ -1551,29 +1554,41 @@ 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 => | ||
cloud-fan marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| if (checksumMismatchFullRetryEnabled) { | ||
| // 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. | ||
| if (stage.isParentIndeterminate) { | ||
| mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) | ||
| sms.shuffleDep.newShuffleMergeState() | ||
| } | ||
| } 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) | ||
| // 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 | ||
| } | ||
| } | ||
| mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) | ||
| sms.shuffleDep.newShuffleMergeState() | ||
| } | ||
| mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) | ||
| sms.shuffleDep.newShuffleMergeState() | ||
| case _ => | ||
| } | ||
|
|
||
|
|
@@ -1886,6 +1901,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 +2051,25 @@ 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 (checksumMismatchFullRetryEnabled && shuffleStage.isStageIndeterminate) { | ||
| abortUnrollbackableStages(shuffleStage) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } else { | ||
| logInfo(log"Ignoring ${MDC(TASK_NAME, smt)} completion from an older attempt of indeterminate stage") | ||
|
|
@@ -2148,12 +2194,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 && !checksumMismatchFullRetryEnabled) { | ||
| abortUnrollbackableStages(mapStage) | ||
| } | ||
|
|
||
| // We expect one executor failure to trigger many FetchFailures in rapid succession, | ||
|
|
||
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.
Does it make sense to use
SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLEDwithoutSCHEDULER_CHECKSUM_MISMATCH_FULL_RETRY_ENABLED? or vice versa?What about getting removing the
SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLED(as the version is where it is introduced is also 4.1.0 we can do that) and computing the checksum whenSCHEDULER_CHECKSUM_MISMATCH_FULL_RETRY_ENABLEDis true? So having only one config for the feature?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 is a good point. Why do we separate the checksum computation and stage retry with two flag? Do we have logging for checksum mismatch without retry?
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.
Yes, there is log written when checksum mismatch happens. If
SCHEDULER_CHECKSUM_MISMATCH_FULL_RETRY_ENABLEDis false, there'll be no fully retry for succeeding stages but only logs for the checksum mismatch.And when we want to enable
SCHEDULER_CHECKSUM_MISMATCH_FULL_RETRY_ENABLED, need to make sureSHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLEDis also true if we keep these two configs.One config can be easier to use. If it makes sense to you all, I'll remove
SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLED.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.
I think it makes sense to have the log-only mode, so that Spark users can do impact analysis before turnning on the retry.
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.
We can improve it a bit more: we will compute checksum when either
SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLEDorSCHEDULER_CHECKSUM_MISMATCH_FULL_RETRY_ENABLEDis enabled.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.
Thanks, updated.