Skip to content
Closed
Show file tree
Hide file tree
Changes from 18 commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -286,7 +286,9 @@ class DAGScheduler(
case None =>
// We are going to register ancestor shuffle dependencies
getAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep =>
shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId)
if (!shuffleToMapStage.contains(dep.shuffleId)) {
shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId)
}
}
// Then register current shuffleDep
val stage = newOrUsedShuffleStage(shuffleDep, firstJobId)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,53 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou
assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1))
}

/**
* This test ensures that DAGScheduler build stage graph correctly.
*
* Suppose you have the following DAG:
*
* [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D]
* \ /
* <-------------
*
* Here, RDD B has a shuffle dependency on RDD A, and RDD C has shuffle dependency on both
* B and A. The shuffle dependency IDs are numbers in the DAGScheduler, but to make the example
* easier to understand, let's call the shuffled data from A shuffle dependency ID s_A and the
* shuffled data from B shuffle dependency ID s_B.
*
* Note: [] means an RDD, () means a shuffle dependency.
*/
test("[SPARK-13902] not to create duplicate stage.") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you change this to "[SPARK-13902] Ensure no duplicate stages are created"?

val rddA = new MyRDD(sc, 1, Nil)
val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1))
val s_A = shuffleDepA.shuffleId

val rddB = new MyRDD(sc, 1, List(shuffleDepA), tracker = mapOutputTracker)
val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1))
val s_B = shuffleDepB.shuffleId

val rddC = new MyRDD(sc, 1, List(shuffleDepA, shuffleDepB), tracker = mapOutputTracker)
val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1))
val s_C = shuffleDepC.shuffleId

val rddD = new MyRDD(sc, 1, List(shuffleDepC), tracker = mapOutputTracker)

submit(rddD, Array(0))

assert(scheduler.shuffleToMapStage.size === 3)
assert(scheduler.activeJobs.size === 1)

val mapStageA = scheduler.shuffleToMapStage(s_A)
val mapStageB = scheduler.shuffleToMapStage(s_B)
val mapStageC = scheduler.shuffleToMapStage(s_C)
val finalStage = scheduler.activeJobs.head.finalStage

assert(mapStageA.parents.isEmpty)
assert(mapStageB.parents === List(mapStageA))
assert(mapStageC.parents === List(mapStageA, mapStageB))
assert(finalStage.parents === List(mapStageC))
}

test("zero split job") {
var numResults = 0
var failureReason: Option[Exception] = None
Expand Down