-
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 6 commits
8e667f5
2bfa05b
d02cf93
7056cd6
1f7bd88
d92adfc
1127ca1
f91d86f
09077cb
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 |
|---|---|---|
|
|
@@ -1235,6 +1235,7 @@ class DAGScheduler( | |
| case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) => | ||
| val failedStage = stageIdToStage(task.stageId) | ||
| val mapStage = shuffleIdToMapStage(shuffleId) | ||
| var abortedStage = false | ||
|
|
||
| if (failedStage.latestInfo.attemptId != task.stageAttemptId) { | ||
| logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + | ||
|
|
@@ -1261,6 +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) | ||
| 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. | ||
|
|
@@ -1271,8 +1273,10 @@ class DAGScheduler( | |
| override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) | ||
| }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) | ||
| } | ||
| failedStages += failedStage | ||
| failedStages += mapStage | ||
| if (!abortedStage) { | ||
| failedStages += failedStage | ||
| failedStages += mapStage | ||
| } | ||
| // Mark the map whose fetch failed as broken in the map stage | ||
| if (mapId != -1) { | ||
| mapStage.removeOutputLoc(mapId, bmAddress) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,9 +18,12 @@ | |
| package org.apache.spark.scheduler | ||
|
|
||
| import java.util.Properties | ||
| import java.util.concurrent.Executors | ||
|
|
||
| import scala.annotation.meta.param | ||
| import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} | ||
| import scala.concurrent.{ExecutionContext, Future} | ||
| import scala.concurrent.duration.DurationConversions | ||
| import scala.language.reflectiveCalls | ||
| import scala.util.control.NonFatal | ||
|
|
||
|
|
@@ -31,9 +34,10 @@ 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.FetchFailedException | ||
|
||
| import org.apache.spark.shuffle.MetadataFetchFailedException | ||
| import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} | ||
| import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} | ||
| import org.apache.spark.util._ | ||
|
|
||
| class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) | ||
| extends DAGSchedulerEventProcessLoop(dagScheduler) { | ||
|
|
@@ -2105,6 +2109,52 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou | |
| assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, shuffleDepC)) | ||
| } | ||
|
|
||
| test("The failed stage never resubmitted due to abort stage in another thread") { | ||
|
||
| implicit val executorContext = ExecutionContext | ||
| .fromExecutorService(Executors.newFixedThreadPool(5)) | ||
| val duration = 60.seconds | ||
|
|
||
| val f1 = Future { | ||
|
||
| try { | ||
| val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 1)).groupByKey() | ||
| val shuffleHandle = | ||
| rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle | ||
| rdd1.map { | ||
| case (x, _) if (x == 1) => | ||
| throw new FetchFailedException( | ||
| BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") | ||
| case (x, _) => x | ||
| }.count() | ||
| } catch { | ||
| case e: Throwable => | ||
|
||
| logInfo("expected abort stage1: " + e.getMessage) | ||
| } | ||
| } | ||
| ThreadUtils.awaitResult(f1, duration) | ||
| val f2 = Future { | ||
|
||
| try { | ||
| val rdd2 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 1)).groupByKey() | ||
| val shuffleHandle = | ||
| rdd2.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle | ||
| rdd2.map { | ||
| case (x, _) if (x == 1) => | ||
| throw new FetchFailedException( | ||
| BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") | ||
| case (x, _) => x | ||
| }.count() | ||
| } catch { | ||
| case e: Throwable => | ||
| logInfo("expected abort stage2: " + e.getMessage) | ||
| } | ||
| } | ||
| try { | ||
| ThreadUtils.awaitResult(f2, duration) | ||
| } catch { | ||
| case e: Throwable => fail("The failed stage never resubmitted") | ||
| } | ||
| executorContext.shutdown() | ||
| } | ||
|
|
||
| /** | ||
| * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. | ||
| * Note that this checks only the host and not the executor ID. | ||
|
|
||
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.