-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17644] [CORE] Do not add failedStages when abortStage for fetch failure #15213
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 1 commit
8e667f5
2bfa05b
d02cf93
7056cd6
1f7bd88
d92adfc
1127ca1
f91d86f
09077cb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1235,7 +1235,7 @@ class DAGScheduler( | |
| case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) => | ||
| val failedStage = stageIdToStage(task.stageId) | ||
| val mapStage = shuffleIdToMapStage(shuffleId) | ||
| var abortStage = false | ||
| var abortedStage = false | ||
|
|
||
| if (failedStage.latestInfo.attemptId != task.stageAttemptId) { | ||
| logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + | ||
|
|
@@ -1262,7 +1262,7 @@ class DAGScheduler( | |
| s"has failed the maximum allowable number of " + | ||
| s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " + | ||
| s"Most recent failure reason: ${failureMessage}", None) | ||
| abortStage = true | ||
| abortedStage = true | ||
|
||
| } 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. | ||
|
|
@@ -1273,7 +1273,7 @@ class DAGScheduler( | |
| override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) | ||
| }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) | ||
| } | ||
| if (!abortStage) { | ||
| if (!abortedStage) { | ||
| failedStages += failedStage | ||
| failedStages += mapStage | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -34,8 +34,8 @@ import org.apache.spark._ | |
| import org.apache.spark.broadcast.BroadcastManager | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.scheduler.SchedulingMode.SchedulingMode | ||
| import org.apache.spark.shuffle.MetadataFetchFailedException | ||
| import org.apache.spark.shuffle.FetchFailedException | ||
|
||
| import org.apache.spark.shuffle.MetadataFetchFailedException | ||
| import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} | ||
| import org.apache.spark.util._ | ||
|
|
||
|
|
@@ -2121,7 +2121,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou | |
| rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle | ||
| rdd1.map { x => | ||
|
||
| if (x._1 == 1) { | ||
| throw new FetchFailedException(BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") | ||
| throw new FetchFailedException( | ||
| BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") | ||
| } | ||
| x._1 | ||
| }.count() | ||
|
|
@@ -2138,7 +2139,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou | |
| rdd2.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle | ||
| rdd2.map { x => | ||
| if (x._1 == 1) { | ||
| throw new FetchFailedException(BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") | ||
| throw new FetchFailedException( | ||
| BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") | ||
| } | ||
| x._1 | ||
| }.count() | ||
|
|
||
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.
Could you move this line just above
if (disallowStageRetryForTest) {since it's only used in that scope.