Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
[SPARK-25211][Core] speculation and fetch failed result in hang of job
  • Loading branch information
liutang123 committed Aug 23, 2018
commit 4f51199daafec0466a5ac836c4f6281f5ba45381
Original file line number Diff line number Diff line change
Expand Up @@ -1402,7 +1402,7 @@ private[spark] class DAGScheduler(
shuffleStage.pendingPartitions -= task.partitionId
}

if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) {
if (shuffleStage.pendingPartitions.isEmpty) {
markStageAsFinished(shuffleStage)
logInfo("looking for newly runnable stages")
logInfo("running: " + runningStages)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.scheduler

import java.util.Properties
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}

import scala.annotation.meta.param
Expand Down Expand Up @@ -2627,6 +2628,39 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assert(countSubmittedMapStageAttempts() === 2)
}

test("SPARK-25211 speculation and fetch failed result in hang of job") {
val shuffleMapRDD1 = new MyRDD(sc, 1, Nil)
val dep1 = new ShuffleDependency(shuffleMapRDD1, new HashPartitioner(2))
val shuffleMapRDD2 = new MyRDD(sc, 2, List(dep1))
val dep2 = new ShuffleDependency(shuffleMapRDD2, new HashPartitioner(2))

val jobId = scheduler.nextJobId.get()
val waiter = new JobWaiter(scheduler, jobId, 1, (_: Int, _: MapOutputStatistics) => {})
val realJobId = submitMapStage(dep2, waiter)
assert(waiter.jobId === realJobId)
assert(taskSets(0).stageId === 0 && taskSets(0).stageAttemptId === 0)
complete(taskSets(0),
(Success, makeMapStatus("hostA", dep1.partitioner.numPartitions)) :: Nil
)
assert(taskSets(1).stageId === 1 && taskSets(1).stageAttemptId === 0)
complete(taskSets(1), Seq(
(Success, makeMapStatus("hostB", dep2.partitioner.numPartitions)),
(FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null)))

// waiting for resubmitting of failed stages
TimeUnit.MILLISECONDS.sleep(DAGScheduler.RESUBMIT_TIMEOUT * 2)
assert(taskSets(2).stageId === 0 && taskSets(2).stageAttemptId === 1)

// A speculated task finished
runEvent(makeCompletionEvent(taskSets(1).tasks(1),
Success, makeMapStatus("hostC", dep2.partitioner.numPartitions)))
assert(waiter.jobFinished)

runEvent(makeCompletionEvent(taskSets(2).tasks(0),
Success, makeMapStatus("hostD", dep2.partitioner.numPartitions)))
assert(taskSets.size === 3)
}

/**
* 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.
Expand Down