From 883c3b3bbca25a8d24f7fa9867d7e42d9f3e8d9a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 16 Aug 2018 02:38:24 +0800 Subject: [PATCH 01/26] fix repartition+shuffle bug --- .../scala/org/apache/spark/Dependency.scala | 10 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 33 ++++++- .../org/apache/spark/rdd/ShuffledRDD.scala | 8 +- .../apache/spark/scheduler/DAGScheduler.scala | 38 ++++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 93 ++++++++++++++++++- .../exchange/ShuffleExchangeExec.scala | 11 ++- 6 files changed, 183 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 9ea6d2fa2fd95..1ced25413cb10 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -65,6 +65,13 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param keyOrdering key ordering for RDD's shuffles * @param aggregator map/reduce-side aggregator for RDD's shuffle * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) + * @param orderSensitivePartitioner whether the partitioner is order sensitive to the input data + * and will partition shuffle output differently if the input data + * order changes. For example, hash and range partitioners are + * order insensitive, round-robin partitioner is order sensitive. + * This is a property of `ShuffleDependency` instead + * of `Partitioner`, because it's common that a map task partitions + * its output by itself and use a dummy partitioner later. */ @DeveloperApi class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( @@ -73,7 +80,8 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val serializer: Serializer = SparkEnv.get.serializer, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false) + val mapSideCombine: Boolean = false, + val orderSensitivePartitioner: Boolean = false) extends Dependency[Product2[K, V]] { if (mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 374b846d2ea57..33200750ccfad 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -112,6 +112,11 @@ abstract class RDD[T: ClassTag]( /** * :: DeveloperApi :: * Implemented by subclasses to compute a given partition. + * + * Spark requires the computing function to always output the same data set(the order can vary) + * given the same input data set. For example, a computing function that increases each input + * value by 1 is valid, a computing function that increases each input by a random value is + * invalid. */ @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] @@ -462,8 +467,11 @@ abstract class RDD[T: ClassTag]( // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition), - new HashPartitioner(numPartitions)), + new ShuffledRDD[Int, T, T]( + mapPartitionsWithIndex(distributePartition), + new HashPartitioner(numPartitions), + // round-robin partitioner is order sensitive. + orderSensitivePartitioner = true), numPartitions, partitionCoalescer).values } else { @@ -853,6 +861,11 @@ abstract class RDD[T: ClassTag]( * second element in each RDD, etc. Assumes that the two RDDs have the *same number of * partitions* and the *same number of elements in each partition* (e.g. one was made through * a map on the other). + * + * Note that, `zip` violates the requirement of the RDD computing function. If the order of input + * data changes, `zip` will return different result. Because of this, Spark may return unexpected + * result if there is a shuffle after `zip`, and the shuffle failed and retried. To workaround + * this, users can call `zipPartitions` and sort the input data before zip. */ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope { zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => @@ -1865,6 +1878,22 @@ abstract class RDD[T: ClassTag]( // RDD chain. @transient protected lazy val isBarrier_ : Boolean = dependencies.filter(!_.isInstanceOf[ShuffleDependency[_, _, _]]).exists(_.rdd.isBarrier()) + + /** + * Whether the RDD's computing function is idempotent. Idempotent means the computing function + * not only satisfies the requirement, but also produce the same output sequence(the output order + * can't vary) given the same input sequence. Spark assumes all the RDDs are idempotent, except + * for the shuffle RDD and RDDs derived from non-idempotent RDD. + */ + // TODO: Add public APIs to allow users to mark their RDD as non-idempotent. + // TODO: this can be per-partition. e.g. UnionRDD can have part of its partitions idempotent. + private[spark] def isIdempotent: Boolean = { + dependencies.forall { dep => + // Shuffle RDD is always considered as non-idempotent, because its computing function needs + // to fetch remote shuffle blocks, and these fetched blocks may arrive in a random order. + !dep.isInstanceOf[ShuffleDependency[_, _, _]] && dep.rdd.isIdempotent + } + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index e8f9b27b7eb55..d0105283002cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -32,6 +32,8 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD + * @param orderSensitivePartitioner whether the partitioner is order sensitive to the input data. + * Please refer to the doc of [[ShuffleDependency]] for details. * @tparam K the key class. * @tparam V the value class. * @tparam C the combiner class. @@ -40,7 +42,8 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @DeveloperApi class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( @transient var prev: RDD[_ <: Product2[K, V]], - part: Partitioner) + part: Partitioner, + orderSensitivePartitioner: Boolean = false) extends RDD[(K, C)](prev.context, Nil) { private var userSpecifiedSerializer: Option[Serializer] = None @@ -84,7 +87,8 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( serializerManager.getSerializer(implicitly[ClassTag[K]], implicitly[ClassTag[V]]) } } - List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) + List(new ShuffleDependency( + prev, part, serializer, keyOrdering, aggregator, mapSideCombine, orderSensitivePartitioner)) } override val partitioner = Some(part) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index fec6558f412d0..0eaf43a25ea79 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1487,6 +1487,44 @@ private[spark] class DAGScheduler( failedStages += failedStage failedStages += mapStage if (noResubmitEnqueued) { + // If the map stage is not idempotent(produces data in a different order when retry) + // and the shuffle partitioner is order sensitive, we have to retry all the tasks of + // the failed stage and its succeeding stages, because the input data of the failed + // stage will be changed after the map tasks are re-tried. + if (!mapStage.rdd.isIdempotent && mapStage.shuffleDep.orderSensitivePartitioner) { + def rollBackStage(stage: Stage): Unit = stage match { + case mapStage: ShuffleMapStage => + if (mapStage.findMissingPartitions().length < mapStage.numPartitions) { + markStageAsFinished( + mapStage, + Some("preceding non-idempotent shuffle map stage gets retried."), + willRetry = !shouldAbortStage) + mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) + failedStages += mapStage + } + + case resultStage => + val numMissingPartitions = resultStage.findMissingPartitions().length + if (numMissingPartitions != resultStage.numPartitions) { + // TODO: support to rollback result tasks. + abortStage(failedStage, "cannot rollback result tasks.", None) + } + } + + def rollbackSucceedingStages(stageChain: List[Stage]): Unit = { + if (stageChain.head.id == failedStage.id) { + stageChain.foreach { stage => + if (!failedStages.contains(stage)) rollBackStage(stage) + } + } else { + stageChain.head.parents.foreach(s => rollbackSucceedingStages(s :: stageChain)) + } + } + + rollBackStage(failedStage) + activeJobs.foreach(job => rollbackSucceedingStages(job.finalStage :: Nil)) + } + // We expect one executor failure to trigger many FetchFailures in rapid succession, // but all of those task failures can typically be handled by a single resubmission of // the failed stage. We avoid flooding the scheduler's event queue with resubmit diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index cd00051c56e8d..0133423352ed5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -70,7 +70,8 @@ class MyRDD( numPartitions: Int, dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil, - @(transient @param) tracker: MapOutputTrackerMaster = null) + @(transient @param) tracker: MapOutputTrackerMaster = null, + idempotent: Boolean = true) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = @@ -80,6 +81,8 @@ class MyRDD( override def index: Int = i }).toArray + override private[spark] def isIdempotent = super.isIdempotent && idempotent + override def getPreferredLocations(partition: Partition): Seq[String] = { if (locations.isDefinedAt(partition.index)) { locations(partition.index) @@ -2633,6 +2636,94 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 2) } + test("SPARK-23207: retry all the succeeding stages when a non-idempotent map stage gets " + + "re-tried via FetchFailed and shuffle partitioner is order sensitive.") { + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, idempotent = false).mapPartitions(iter => iter) + assert(!shuffleMapRdd1.isIdempotent) + val shuffleDep1 = new ShuffleDependency( + shuffleMapRdd1, new HashPartitioner(2), orderSensitivePartitioner = true) + val shuffleId1 = shuffleDep1.shuffleId + + val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker) + assert(!shuffleMapRdd2.isIdempotent) + val shuffleDep2 = new ShuffleDependency( + shuffleMapRdd2, new HashPartitioner(2), orderSensitivePartitioner = false) + val shuffleId2 = shuffleDep2.shuffleId + + val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) + submit(finalRdd, Array(0, 1)) + + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + complete(taskSets(1), Seq( + (Success, makeMapStatus("hostC", 2)), + (Success, makeMapStatus("hostD", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + + // The first task of the final stage succeeds + runEvent(makeCompletionEvent( + taskSets(2).tasks(0), Success, 42, + Seq.empty, createFakeTaskInfoWithId(0))) + + // The second task of the final stage failed with fetch failure + runEvent(makeCompletionEvent( + taskSets(2).tasks(1), + FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), + null)) + + var failedStages = scheduler.failedStages.toSeq + assert(failedStages.length == 2) + // Shuffle blocks of "hostC" is lost, so first task of `shuffleMapRdd2` needs to retry. + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage + }.head.findMissingPartitions() == Seq(0)) + // The result stage is waiting for the second task to complete. We don't need to rollback this + // stage because the partitioner of shuffle 2 is order insensitive. + val resultStage = failedStages.collect { + case stage: ResultStage => stage + }.head + assert(resultStage.findMissingPartitions() == Seq(1)) + + scheduler.resubmitFailedStages() + // reset the final stage, as currently DAGScheduler can't rollback result stage + resultStage.activeJob.get.resetAllPartitions() + + // The first task of the second shuffle map stage failed with fetch failure + runEvent(makeCompletionEvent( + taskSets(3).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), + null)) + + failedStages = scheduler.failedStages.toSeq + assert(failedStages.length == 2) + // Shuffle blocks of "hostA" is lost, so first task of first shuffle map stage needs to retry. + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId1 => stage + }.head.findMissingPartitions() == Seq(0)) + // The second shuffle map stage is entirely rollbacked, because the partitioner of shuffle 1 is + // order sensitive. + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage + }.head.findMissingPartitions() == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // complete the first task of first shuffle map stage + runEvent(makeCompletionEvent( + taskSets(0).tasks(0), Success, makeMapStatus("hostA", 2))) + // Complete the map stage. + completeShuffleMapStageSuccessfully(0, 1, numShufflePartitions = 2) + completeShuffleMapStageSuccessfully(1, 2, numShufflePartitions = 2) + // Complete the result stage. + completeNextResultStageWithSuccess(2, 1) + + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assertDataStructuresEmpty() + } + /** * 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. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 50f10c31427d0..822686d1467ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -258,6 +258,9 @@ object ShuffleExchangeExec { case _ => sys.error(s"Exchange not implemented for $newPartitioning") } + val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] && + newPartitioning.numPartitions > 1 + val rddWithPartitionIds: RDD[Product2[Int, InternalRow]] = { // [SPARK-23207] Have to make sure the generated RoundRobinPartitioning is deterministic, // otherwise a retry task may output different rows and thus lead to data loss. @@ -267,9 +270,7 @@ object ShuffleExchangeExec { // // Note that we don't perform local sort if the new partitioning has only 1 partition, under // that case all output rows go to the same partition. - val newRdd = if (SQLConf.get.sortBeforeRepartition && - newPartitioning.numPartitions > 1 && - newPartitioning.isInstanceOf[RoundRobinPartitioning]) { + val newRdd = if (isRoundRobin && SQLConf.get.sortBeforeRepartition) { rdd.mapPartitionsInternal { iter => val recordComparatorSupplier = new Supplier[RecordComparator] { override def get: RecordComparator = new RecordBinaryComparator() @@ -326,7 +327,9 @@ object ShuffleExchangeExec { new ShuffleDependency[Int, InternalRow, InternalRow]( rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), - serializer) + serializer, + // round-robin partitioner is order sensitive if we don't sort the input. + orderSensitivePartitioner = isRoundRobin && !SQLConf.get.sortBeforeRepartition) dependency } From 0f815d0afce588932660d607479ed6ca95a74051 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 17 Aug 2018 01:07:46 +0800 Subject: [PATCH 02/26] fix mima --- .../scala/org/apache/spark/Dependency.scala | 20 ++++++++++--------- .../org/apache/spark/rdd/ShuffledRDD.scala | 10 +++++++--- .../spark/scheduler/DAGSchedulerSuite.scala | 8 ++++---- .../exchange/ShuffleExchangeExec.scala | 6 +++--- 4 files changed, 25 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 1ced25413cb10..524128554ba5c 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -65,13 +65,6 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param keyOrdering key ordering for RDD's shuffles * @param aggregator map/reduce-side aggregator for RDD's shuffle * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) - * @param orderSensitivePartitioner whether the partitioner is order sensitive to the input data - * and will partition shuffle output differently if the input data - * order changes. For example, hash and range partitioners are - * order insensitive, round-robin partitioner is order sensitive. - * This is a property of `ShuffleDependency` instead - * of `Partitioner`, because it's common that a map task partitions - * its output by itself and use a dummy partitioner later. */ @DeveloperApi class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( @@ -80,8 +73,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val serializer: Serializer = SparkEnv.get.serializer, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false, - val orderSensitivePartitioner: Boolean = false) + val mapSideCombine: Boolean = false) extends Dependency[Product2[K, V]] { if (mapSideCombine) { @@ -102,6 +94,16 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( shuffleId, _rdd.partitions.length, this) _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) + + /** + * whether the partitioner is order sensitive to the input data and will partition shuffle output + * differently if the input data order changes. For example, hash and range partitioners are + * order insensitive, round-robin partitioner is order sensitive. This is a property of + * `ShuffleDependency` instead of `Partitioner`, because it's common that a map task partitions + * its output by itself and use a dummy partitioner later. + */ + // This is defined as a `var` here instead of the constructor, to pass the mima check. + private[spark] var orderSensitivePartitioner: Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index d0105283002cf..cf5b46970a05b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -43,9 +43,11 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( @transient var prev: RDD[_ <: Product2[K, V]], part: Partitioner, - orderSensitivePartitioner: Boolean = false) + orderSensitivePartitioner: Boolean) extends RDD[(K, C)](prev.context, Nil) { + def this(prev: RDD[_ <: Product2[K, V]], part: Partitioner) = this(prev, part, false) + private var userSpecifiedSerializer: Option[Serializer] = None private var keyOrdering: Option[Ordering[K]] = None @@ -87,8 +89,10 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( serializerManager.getSerializer(implicitly[ClassTag[K]], implicitly[ClassTag[V]]) } } - List(new ShuffleDependency( - prev, part, serializer, keyOrdering, aggregator, mapSideCombine, orderSensitivePartitioner)) + val dep = new ShuffleDependency( + prev, part, serializer, keyOrdering, aggregator, mapSideCombine) + dep.orderSensitivePartitioner = orderSensitivePartitioner + List(dep) } override val partitioner = Some(part) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 0133423352ed5..bf1a4dfc9c4c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2640,14 +2640,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi "re-tried via FetchFailed and shuffle partitioner is order sensitive.") { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, idempotent = false).mapPartitions(iter => iter) assert(!shuffleMapRdd1.isIdempotent) - val shuffleDep1 = new ShuffleDependency( - shuffleMapRdd1, new HashPartitioner(2), orderSensitivePartitioner = true) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) + shuffleDep1.orderSensitivePartitioner = true val shuffleId1 = shuffleDep1.shuffleId val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker) assert(!shuffleMapRdd2.isIdempotent) - val shuffleDep2 = new ShuffleDependency( - shuffleMapRdd2, new HashPartitioner(2), orderSensitivePartitioner = false) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) + shuffleDep2.orderSensitivePartitioner = false val shuffleId2 = shuffleDep2.shuffleId val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 822686d1467ba..b6ebf8a4eee4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -327,9 +327,9 @@ object ShuffleExchangeExec { new ShuffleDependency[Int, InternalRow, InternalRow]( rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), - serializer, - // round-robin partitioner is order sensitive if we don't sort the input. - orderSensitivePartitioner = isRoundRobin && !SQLConf.get.sortBeforeRepartition) + serializer) + // round-robin partitioner is order sensitive if we don't sort the input. + dependency.orderSensitivePartitioner = isRoundRobin && !SQLConf.get.sortBeforeRepartition dependency } From 5aaec486b951627f29d22d60885dcea96a76a9d7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 17 Aug 2018 23:10:15 +0800 Subject: [PATCH 03/26] fix doc --- core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index cf5b46970a05b..2a7e0efa5edc0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -33,7 +33,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @param orderSensitivePartitioner whether the partitioner is order sensitive to the input data. - * Please refer to the doc of [[ShuffleDependency]] for details. + * Please refer to the doc of `ShuffleDependency` for details. * @tparam K the key class. * @tparam V the value class. * @tparam C the combiner class. From ca3c369ec859fa168498bbcf5cd4d16065213a6e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Aug 2018 16:13:19 +0800 Subject: [PATCH 04/26] better abstraction --- .../scala/org/apache/spark/Dependency.scala | 10 -- .../scala/org/apache/spark/Partitioner.scala | 3 + .../apache/spark/rdd/MapPartitionsRDD.scala | 14 ++- .../main/scala/org/apache/spark/rdd/RDD.scala | 102 +++++++++++------- .../org/apache/spark/rdd/ShuffledRDD.scala | 12 +-- .../spark/rdd/ZippedPartitionsRDD.scala | 18 +++- .../apache/spark/scheduler/DAGScheduler.scala | 18 ++-- .../exchange/ShuffleExchangeExec.scala | 12 +-- 8 files changed, 118 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 524128554ba5c..9ea6d2fa2fd95 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -94,16 +94,6 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( shuffleId, _rdd.partitions.length, this) _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) - - /** - * whether the partitioner is order sensitive to the input data and will partition shuffle output - * differently if the input data order changes. For example, hash and range partitioners are - * order insensitive, round-robin partitioner is order sensitive. This is a property of - * `ShuffleDependency` instead of `Partitioner`, because it's common that a map task partitions - * its output by itself and use a dummy partitioner later. - */ - // This is defined as a `var` here instead of the constructor, to pass the mima check. - private[spark] var orderSensitivePartitioner: Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index c940cb25d478b..fdb5df21ac9ff 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -33,6 +33,9 @@ import org.apache.spark.util.random.SamplingUtils /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. + * + * Note that, partitioner must be idempotent, i.e. it must return the same partition id given the + * same partition key. */ abstract class Partitioner extends Serializable { def numPartitions: Int diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 904d9c025629f..42bfd23d58953 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -32,12 +32,16 @@ import org.apache.spark.{Partition, TaskContext} * doesn't modify the keys. * @param isFromBarrier Indicates whether this RDD is transformed from an RDDBarrier, a stage * containing at least one RDDBarrier shall be turned into a barrier stage. + * @param orderSensitiveFunc whether or not the zip function is order-sensitive. If it's order + * sensitive, it may return totally different result if the input order + * changed. Mostly stateful functions are order-sensitive. */ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( var prev: RDD[T], f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false, - isFromBarrier: Boolean = false) + isFromBarrier: Boolean = false, + orderSensitiveFunc: Boolean = false) extends RDD[U](prev) { override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None @@ -54,4 +58,12 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( @transient protected lazy override val isBarrier_ : Boolean = isFromBarrier || dependencies.exists(_.rdd.isBarrier()) + + override private[spark] def computingRandomLevel = { + if (orderSensitiveFunc && prev.computingRandomLevel == RDD.RandomLevel.RANDOM_ORDER) { + RDD.RandomLevel.COMPLETE_RANDOM + } else { + super.computingRandomLevel + } + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 33200750ccfad..0ffcddaa729fd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -112,11 +112,6 @@ abstract class RDD[T: ClassTag]( /** * :: DeveloperApi :: * Implemented by subclasses to compute a given partition. - * - * Spark requires the computing function to always output the same data set(the order can vary) - * given the same input data set. For example, a computing function that increases each input - * value by 1 is valid, a computing function that increases each input by a random value is - * invalid. */ @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] @@ -468,10 +463,8 @@ abstract class RDD[T: ClassTag]( // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( new ShuffledRDD[Int, T, T]( - mapPartitionsWithIndex(distributePartition), - new HashPartitioner(numPartitions), - // round-robin partitioner is order sensitive. - orderSensitivePartitioner = true), + mapPartitionsWithIndexInternal(distributePartition, orderSensitiveFunc = true), + new HashPartitioner(numPartitions)), numPartitions, partitionCoalescer).values } else { @@ -820,11 +813,13 @@ abstract class RDD[T: ClassTag]( */ private[spark] def mapPartitionsWithIndexInternal[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean = false): RDD[U] = withScope { + preservesPartitioning: Boolean = false, + orderSensitiveFunc: Boolean = false): RDD[U] = withScope { new MapPartitionsRDD( this, (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), - preservesPartitioning) + preservesPartitioning, + orderSensitiveFunc) } /** @@ -861,23 +856,19 @@ abstract class RDD[T: ClassTag]( * second element in each RDD, etc. Assumes that the two RDDs have the *same number of * partitions* and the *same number of elements in each partition* (e.g. one was made through * a map on the other). - * - * Note that, `zip` violates the requirement of the RDD computing function. If the order of input - * data changes, `zip` will return different result. Because of this, Spark may return unexpected - * result if there is a shuffle after `zip`, and the shuffle failed and retried. To workaround - * this, users can call `zipPartitions` and sort the input data before zip. */ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope { - zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => - new Iterator[(T, U)] { - def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match { - case (true, true) => true - case (false, false) => false - case _ => throw new SparkException("Can only zip RDDs with " + - "same number of elements in each partition") + zipPartitionsInternal(other, preservesPartitioning = false, orderSensitiveFunc = true) { + (thisIter, otherIter) => + new Iterator[(T, U)] { + def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match { + case (true, true) => true + case (false, false) => false + case _ => throw new SparkException("Can only zip RDDs with " + + "same number of elements in each partition") + } + def next(): (T, U) = (thisIter.next(), otherIter.next()) } - def next(): (T, U) = (thisIter.next(), otherIter.next()) - } } } @@ -893,6 +884,13 @@ abstract class RDD[T: ClassTag]( new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning) } + private[spark] def zipPartitionsInternal[B: ClassTag, V: ClassTag] + (rdd2: RDD[B], preservesPartitioning: Boolean, orderSensitiveFunc: Boolean) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope { + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning, orderSensitiveFunc) + } + + def zipPartitions[B: ClassTag, V: ClassTag] (rdd2: RDD[B]) (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope { @@ -1880,18 +1878,35 @@ abstract class RDD[T: ClassTag]( dependencies.filter(!_.isInstanceOf[ShuffleDependency[_, _, _]]).exists(_.rdd.isBarrier()) /** - * Whether the RDD's computing function is idempotent. Idempotent means the computing function - * not only satisfies the requirement, but also produce the same output sequence(the output order - * can't vary) given the same input sequence. Spark assumes all the RDDs are idempotent, except - * for the shuffle RDD and RDDs derived from non-idempotent RDD. + * Returns the random level of this RDD's computing function. Please refer to [[RDD.RandomLevel]] + * for the definition of random level. + * + * By default, an RDD without parents(root RDD) is IDEMPOTENT. For RDDs with parents, the random + * level of current RDD is the random level of the parent which is random most. */ - // TODO: Add public APIs to allow users to mark their RDD as non-idempotent. - // TODO: this can be per-partition. e.g. UnionRDD can have part of its partitions idempotent. - private[spark] def isIdempotent: Boolean = { - dependencies.forall { dep => - // Shuffle RDD is always considered as non-idempotent, because its computing function needs - // to fetch remote shuffle blocks, and these fetched blocks may arrive in a random order. - !dep.isInstanceOf[ShuffleDependency[_, _, _]] && dep.rdd.isIdempotent + // TODO: make it public so users can set random level to their custom RDDs. + // TODO: this can be per-partition. e.g. UnionRDD can have different random level for different + // partitions. + private[spark] def computingRandomLevel: RDD.RandomLevel.Value = { + val parentRandomLevels = dependencies.map { + case dep: ShuffleDependency[_, _, _] => + if (dep.rdd.computingRandomLevel == RDD.RandomLevel.COMPLETE_RANDOM) { + RDD.RandomLevel.COMPLETE_RANDOM + } else { + // In Spark, the reducer fetches multiple remote shuffle blocks at the same time, and + // the arrival order of these shuffle blocks are totally random. Which means, the + // computing function of a shuffled RDD will never be "NO_RANDOM" + RDD.RandomLevel.RANDOM_ORDER + } + + case dep => dep.rdd.computingRandomLevel + } + + if (parentRandomLevels.isEmpty) { + // By default we assume the root RDD is idempotent + RDD.RandomLevel.IDEMPOTENT + } else { + parentRandomLevels.maxBy(_.id) } } } @@ -1905,6 +1920,21 @@ abstract class RDD[T: ClassTag]( */ object RDD { + /** + * The random level of RDD's computing function, which indicates the behavior when rerun the + * computing function. There are 3 random levels, ordered by the randomness from low to high: + * 1. IDEMPOTENT. The computing function always return the same result with same order when rerun. + * 2. RANDOM_ORDER. The computing function returns same data set in random order when rerun. + * 3. COMPLETE_RANDOM. The computing function returns different result when rerun. + * + * Note that, the output of the computing function usually relies on parent RDDs. When a + * parent RDD's computing function is random, it's very likely this computing function is also + * random. + */ + object RandomLevel extends Enumeration { + val IDEMPOTENT, RANDOM_ORDER, COMPLETE_RANDOM = Value + } + private[spark] val CHECKPOINT_ALL_MARKED_ANCESTORS = "spark.checkpoint.checkpointAllMarkedAncestors" diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 2a7e0efa5edc0..e8f9b27b7eb55 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -32,8 +32,6 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD - * @param orderSensitivePartitioner whether the partitioner is order sensitive to the input data. - * Please refer to the doc of `ShuffleDependency` for details. * @tparam K the key class. * @tparam V the value class. * @tparam C the combiner class. @@ -42,12 +40,9 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @DeveloperApi class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( @transient var prev: RDD[_ <: Product2[K, V]], - part: Partitioner, - orderSensitivePartitioner: Boolean) + part: Partitioner) extends RDD[(K, C)](prev.context, Nil) { - def this(prev: RDD[_ <: Product2[K, V]], part: Partitioner) = this(prev, part, false) - private var userSpecifiedSerializer: Option[Serializer] = None private var keyOrdering: Option[Ordering[K]] = None @@ -89,10 +84,7 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( serializerManager.getSerializer(implicitly[ClassTag[K]], implicitly[ClassTag[V]]) } } - val dep = new ShuffleDependency( - prev, part, serializer, keyOrdering, aggregator, mapSideCombine) - dep.orderSensitivePartitioner = orderSensitivePartitioner - List(dep) + List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) } override val partitioner = Some(part) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 3cb1231bd3477..2d0a20e79d636 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -81,7 +81,11 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] var f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], - preservesPartitioning: Boolean = false) + preservesPartitioning: Boolean = false, + // whether or not the zip function is order-sensitive. If it's order sensitive, it may return + // totally different result if the input order changed. As an example, `iter1.zip(iter2)` is + // order sensitive. + orderSensitiveFunc: Boolean = false) extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { @@ -95,6 +99,18 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] rdd2 = null f = null } + + private def isRandomOrder(rdd: RDD[_]): Boolean = { + rdd.computingRandomLevel == RDD.RandomLevel.RANDOM_ORDER + } + + override private[spark] def computingRandomLevel = { + if (orderSensitiveFunc && (isRandomOrder(rdd1) || isRandomOrder(rdd2))) { + RDD.RandomLevel.COMPLETE_RANDOM + } else { + super.computingRandomLevel + } + } } private[spark] class ZippedPartitionsRDD3 diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0eaf43a25ea79..e8358a85dde02 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1487,14 +1487,18 @@ private[spark] class DAGScheduler( failedStages += failedStage failedStages += mapStage if (noResubmitEnqueued) { - // If the map stage is not idempotent(produces data in a different order when retry) - // and the shuffle partitioner is order sensitive, we have to retry all the tasks of - // the failed stage and its succeeding stages, because the input data of the failed - // stage will be changed after the map tasks are re-tried. - if (!mapStage.rdd.isIdempotent && mapStage.shuffleDep.orderSensitivePartitioner) { + // If the map stage is COMPLETE_RANDOM, which means the map tasks will return + // different result when re-try, we need to re-try all the tasks of the failed + // stage and its succeeding stages, because the input data will be changed after the + // map tasks are re-tried. + // Note that, if map stage is RANDOM_ORDER, we are fine. The shuffle partitioner is + // guaranteed to be idempotent, so the input data will not change even if the map + // tasks are not re-tried. + if (mapStage.rdd.computingRandomLevel == RDD.RandomLevel.COMPLETE_RANDOM) { def rollBackStage(stage: Stage): Unit = stage match { case mapStage: ShuffleMapStage => - if (mapStage.findMissingPartitions().length < mapStage.numPartitions) { + val numMissingPartitions = mapStage.findMissingPartitions().length + if (numMissingPartitions < mapStage.numTasks) { markStageAsFinished( mapStage, Some("preceding non-idempotent shuffle map stage gets retried."), @@ -1505,7 +1509,7 @@ private[spark] class DAGScheduler( case resultStage => val numMissingPartitions = resultStage.findMissingPartitions().length - if (numMissingPartitions != resultStage.numPartitions) { + if (numMissingPartitions != resultStage.numTasks) { // TODO: support to rollback result tasks. abortStage(failedStage, "cannot rollback result tasks.", None) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index b6ebf8a4eee4a..5fcbce98aaa34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -306,17 +306,19 @@ object ShuffleExchangeExec { rdd } + // round-robin function is order sensitive if we don't sort the input. + val orderSensitiveFunc = isRoundRobin && !SQLConf.get.sortBeforeRepartition if (needToCopyObjectsBeforeShuffle(part)) { - newRdd.mapPartitionsInternal { iter => + newRdd.mapPartitionsWithIndexInternal((_, iter) => { val getPartitionKey = getPartitionKeyExtractor() iter.map { row => (part.getPartition(getPartitionKey(row)), row.copy()) } - } + }, orderSensitiveFunc = orderSensitiveFunc) } else { - newRdd.mapPartitionsInternal { iter => + newRdd.mapPartitionsWithIndexInternal((_, iter) => { val getPartitionKey = getPartitionKeyExtractor() val mutablePair = new MutablePair[Int, InternalRow]() iter.map { row => mutablePair.update(part.getPartition(getPartitionKey(row)), row) } - } + }, orderSensitiveFunc = orderSensitiveFunc) } } @@ -328,8 +330,6 @@ object ShuffleExchangeExec { rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), serializer) - // round-robin partitioner is order sensitive if we don't sort the input. - dependency.orderSensitivePartitioner = isRoundRobin && !SQLConf.get.sortBeforeRepartition dependency } From b45b9849d6b95ba0dc105a3cb47ea0072a90804b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Aug 2018 23:01:32 +0800 Subject: [PATCH 05/26] better abstraction --- .../main/scala/org/apache/spark/rdd/RDD.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 40 +++++++------------ 2 files changed, 16 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 0ffcddaa729fd..6ef101730b5df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -818,8 +818,8 @@ abstract class RDD[T: ClassTag]( new MapPartitionsRDD( this, (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), - preservesPartitioning, - orderSensitiveFunc) + preservesPartitioning = preservesPartitioning, + orderSensitiveFunc = orderSensitiveFunc) } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index bf1a4dfc9c4c4..0adad0494271e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -71,7 +71,7 @@ class MyRDD( dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil, @(transient @param) tracker: MapOutputTrackerMaster = null, - idempotent: Boolean = true) + isRandom: Boolean = false) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = @@ -81,7 +81,9 @@ class MyRDD( override def index: Int = i }).toArray - override private[spark] def isIdempotent = super.isIdempotent && idempotent + override private[spark] def computingRandomLevel = { + if (isRandom) RDD.RandomLevel.COMPLETE_RANDOM else super.computingRandomLevel + } override def getPreferredLocations(partition: Partition): Seq[String] = { if (locations.isDefinedAt(partition.index)) { @@ -2636,18 +2638,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 2) } - test("SPARK-23207: retry all the succeeding stages when a non-idempotent map stage gets " + - "re-tried via FetchFailed and shuffle partitioner is order sensitive.") { - val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, idempotent = false).mapPartitions(iter => iter) - assert(!shuffleMapRdd1.isIdempotent) + test("SPARK-23207: retry all the succeeding stages when the map stage is random") { + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, isRandom = true) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) - shuffleDep1.orderSensitivePartitioner = true val shuffleId1 = shuffleDep1.shuffleId val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker) - assert(!shuffleMapRdd2.isIdempotent) val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) - shuffleDep2.orderSensitivePartitioner = false val shuffleId2 = shuffleDep2.shuffleId val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) @@ -2663,35 +2660,27 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostD", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) - // The first task of the final stage succeeds - runEvent(makeCompletionEvent( - taskSets(2).tasks(0), Success, 42, - Seq.empty, createFakeTaskInfoWithId(0))) - - // The second task of the final stage failed with fetch failure + // The first task of the final stage failed with fetch failure runEvent(makeCompletionEvent( - taskSets(2).tasks(1), + taskSets(2).tasks(0), FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), null)) var failedStages = scheduler.failedStages.toSeq assert(failedStages.length == 2) - // Shuffle blocks of "hostC" is lost, so first task of `shuffleMapRdd2` needs to retry. + // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0)) - // The result stage is waiting for the second task to complete. We don't need to rollback this - // stage because the partitioner of shuffle 2 is order insensitive. + // The result stage is still waiting for its 2 tasks to complete val resultStage = failedStages.collect { case stage: ResultStage => stage }.head - assert(resultStage.findMissingPartitions() == Seq(1)) + assert(resultStage.findMissingPartitions() == Seq(0, 1)) scheduler.resubmitFailedStages() - // reset the final stage, as currently DAGScheduler can't rollback result stage - resultStage.activeJob.get.resetAllPartitions() - // The first task of the second shuffle map stage failed with fetch failure + // The first task of the `shuffleMapRdd2` failed with fetch failure runEvent(makeCompletionEvent( taskSets(3).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), @@ -2699,12 +2688,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi failedStages = scheduler.failedStages.toSeq assert(failedStages.length == 2) - // Shuffle blocks of "hostA" is lost, so first task of first shuffle map stage needs to retry. + // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd1` needs to retry. assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId1 => stage }.head.findMissingPartitions() == Seq(0)) - // The second shuffle map stage is entirely rollbacked, because the partitioner of shuffle 1 is - // order sensitive. + // The second shuffle map stage is entirely rollbacked, because the root RDD is random. assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0, 1)) From e92177b590238f4612528ad86ba44de8c3b4daf1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 22 Aug 2018 15:43:31 +0800 Subject: [PATCH 06/26] improve error message --- .../org/apache/spark/scheduler/DAGScheduler.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e8358a85dde02..605bf2a440b12 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1492,8 +1492,8 @@ private[spark] class DAGScheduler( // stage and its succeeding stages, because the input data will be changed after the // map tasks are re-tried. // Note that, if map stage is RANDOM_ORDER, we are fine. The shuffle partitioner is - // guaranteed to be idempotent, so the input data will not change even if the map - // tasks are not re-tried. + // guaranteed to be idempotent, so the input data of the reducers will not change even + // if the map tasks are not re-tried. if (mapStage.rdd.computingRandomLevel == RDD.RandomLevel.COMPLETE_RANDOM) { def rollBackStage(stage: Stage): Unit = stage match { case mapStage: ShuffleMapStage => @@ -1501,7 +1501,7 @@ private[spark] class DAGScheduler( if (numMissingPartitions < mapStage.numTasks) { markStageAsFinished( mapStage, - Some("preceding non-idempotent shuffle map stage gets retried."), + Some("preceding shuffle map stage with random output gets retried."), willRetry = !shouldAbortStage) mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) failedStages += mapStage @@ -1509,9 +1509,14 @@ private[spark] class DAGScheduler( case resultStage => val numMissingPartitions = resultStage.findMissingPartitions().length - if (numMissingPartitions != resultStage.numTasks) { + if (numMissingPartitions < resultStage.numTasks) { // TODO: support to rollback result tasks. - abortStage(failedStage, "cannot rollback result tasks.", None) + val errorMessage = "A shuffle map stage with random output was failed and " + + s"retried. However, Spark cannot rollback the result stage $resultStage " + + "to re-process the input data, and has to fail this job. Please " + + "eliminate the randomness by checkpointing the RDD before " + + "repartition/zip and try again." + abortStage(failedStage, errorMessage, None) } } From 4840ee26b0d6ae53f8805ff3189855c58ad426a7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 22 Aug 2018 22:32:39 +0800 Subject: [PATCH 07/26] allow overwrite task --- .../apache/spark/scheduler/DAGScheduler.scala | 55 ++++++++++++------- .../spark/scheduler/DAGSchedulerEvent.scala | 3 +- .../apache/spark/scheduler/ResultStage.scala | 3 +- 3 files changed, 40 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 605bf2a440b12..7963a7b8f7382 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -441,13 +441,15 @@ private[spark] class DAGScheduler( func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], jobId: Int, - callSite: CallSite): ResultStage = { + callSite: CallSite, + taskOverwritable: Boolean): ResultStage = { checkBarrierStageWithDynamicAllocation(rdd) checkBarrierStageWithNumSlots(rdd) checkBarrierStageWithRDDChainPattern(rdd, partitions.toSet.size) val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite) + val stage = new ResultStage( + id, rdd, func, partitions, parents, jobId, callSite, taskOverwritable) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage @@ -678,7 +680,8 @@ private[spark] class DAGScheduler( partitions: Seq[Int], callSite: CallSite, resultHandler: (Int, U) => Unit, - properties: Properties): JobWaiter[U] = { + properties: Properties, + taskOverwritable: Boolean = false): JobWaiter[U] = { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length partitions.find(p => p >= maxPartitions || p < 0).foreach { p => @@ -698,7 +701,7 @@ private[spark] class DAGScheduler( val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler) eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, partitions.toArray, callSite, waiter, - SerializationUtils.clone(properties))) + SerializationUtils.clone(properties), taskOverwritable)) waiter } @@ -722,9 +725,11 @@ private[spark] class DAGScheduler( partitions: Seq[Int], callSite: CallSite, resultHandler: (Int, U) => Unit, - properties: Properties): Unit = { + properties: Properties, + taskOverwritable: Boolean = false): Unit = { val start = System.nanoTime - val waiter = submitJob(rdd, func, partitions, callSite, resultHandler, properties) + val waiter = submitJob( + rdd, func, partitions, callSite, resultHandler, properties, taskOverwritable) ThreadUtils.awaitReady(waiter.completionFuture, Duration.Inf) waiter.completionFuture.value.get match { case scala.util.Success(_) => @@ -954,12 +959,13 @@ private[spark] class DAGScheduler( partitions: Array[Int], callSite: CallSite, listener: JobListener, - properties: Properties) { + properties: Properties, + taskOverwritable: Boolean) { var finalStage: ResultStage = null try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite) + finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite, taskOverwritable) } catch { case e: BarrierJobSlotsNumberCheckFailed => logWarning(s"The job $jobId requires to run a barrier stage that requires more slots " + @@ -973,7 +979,7 @@ private[spark] class DAGScheduler( messageScheduler.schedule( new Runnable { override def run(): Unit = eventProcessLoop.post(JobSubmitted(jobId, finalRDD, func, - partitions, callSite, listener, properties)) + partitions, callSite, listener, properties, taskOverwritable)) }, timeIntervalNumTasksCheck, TimeUnit.SECONDS @@ -1507,16 +1513,19 @@ private[spark] class DAGScheduler( failedStages += mapStage } - case resultStage => + case resultStage: ResultStage => val numMissingPartitions = resultStage.findMissingPartitions().length if (numMissingPartitions < resultStage.numTasks) { - // TODO: support to rollback result tasks. - val errorMessage = "A shuffle map stage with random output was failed and " + - s"retried. However, Spark cannot rollback the result stage $resultStage " + - "to re-process the input data, and has to fail this job. Please " + - "eliminate the randomness by checkpointing the RDD before " + - "repartition/zip and try again." - abortStage(failedStage, errorMessage, None) + if (resultStage.taskOverwritable) { + resultStage.activeJob.foreach(_.resetAllPartitions()) + } else { + val errorMessage = "A shuffle map stage with random output was failed " + + "and retried. However, Spark cannot rollback the result stage " + + s"$resultStage to re-process the input data, and has to fail this job. " + + "Please eliminate the randomness by checkpointing the RDD before " + + "repartition/zip and try again." + abortStage(failedStage, errorMessage, None) + } } } @@ -2043,8 +2052,16 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler } private def doOnReceive(event: DAGSchedulerEvent): Unit = event match { - case JobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) => - dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) + case j: JobSubmitted => + dagScheduler.handleJobSubmitted( + j.jobId, + j.finalRDD, + j.func, + j.partitions, + j.callSite, + j.listener, + j.properties, + j.taskOverwritable) case MapStageSubmitted(jobId, dependency, callSite, listener, properties) => dagScheduler.handleMapStageSubmitted(jobId, dependency, callSite, listener, properties) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 54ab8f8b3e1d8..018def04ca16a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -41,7 +41,8 @@ private[scheduler] case class JobSubmitted( partitions: Array[Int], callSite: CallSite, listener: JobListener, - properties: Properties = null) + properties: Properties = null, + taskOverwritable: Boolean = false) extends DAGSchedulerEvent /** A map stage as submitted to run as a separate job */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala index d1687830ff7bf..ef4971eff3db7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -34,7 +34,8 @@ private[spark] class ResultStage( val partitions: Array[Int], parents: List[Stage], firstJobId: Int, - callSite: CallSite) + callSite: CallSite, + val taskOverwritable: Boolean) extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite) { /** From 9291c62c1d3337a031ff55ef98d07a0bc347cc6a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Aug 2018 08:34:48 +0800 Subject: [PATCH 08/26] Revert "allow overwrite task" This reverts commit 3a901b9ac0eed30aa7c979941da64eeab47c707e. --- .../apache/spark/scheduler/DAGScheduler.scala | 55 +++++++------------ .../spark/scheduler/DAGSchedulerEvent.scala | 3 +- .../apache/spark/scheduler/ResultStage.scala | 3 +- 3 files changed, 21 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7963a7b8f7382..605bf2a440b12 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -441,15 +441,13 @@ private[spark] class DAGScheduler( func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], jobId: Int, - callSite: CallSite, - taskOverwritable: Boolean): ResultStage = { + callSite: CallSite): ResultStage = { checkBarrierStageWithDynamicAllocation(rdd) checkBarrierStageWithNumSlots(rdd) checkBarrierStageWithRDDChainPattern(rdd, partitions.toSet.size) val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - val stage = new ResultStage( - id, rdd, func, partitions, parents, jobId, callSite, taskOverwritable) + val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage @@ -680,8 +678,7 @@ private[spark] class DAGScheduler( partitions: Seq[Int], callSite: CallSite, resultHandler: (Int, U) => Unit, - properties: Properties, - taskOverwritable: Boolean = false): JobWaiter[U] = { + properties: Properties): JobWaiter[U] = { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length partitions.find(p => p >= maxPartitions || p < 0).foreach { p => @@ -701,7 +698,7 @@ private[spark] class DAGScheduler( val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler) eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, partitions.toArray, callSite, waiter, - SerializationUtils.clone(properties), taskOverwritable)) + SerializationUtils.clone(properties))) waiter } @@ -725,11 +722,9 @@ private[spark] class DAGScheduler( partitions: Seq[Int], callSite: CallSite, resultHandler: (Int, U) => Unit, - properties: Properties, - taskOverwritable: Boolean = false): Unit = { + properties: Properties): Unit = { val start = System.nanoTime - val waiter = submitJob( - rdd, func, partitions, callSite, resultHandler, properties, taskOverwritable) + val waiter = submitJob(rdd, func, partitions, callSite, resultHandler, properties) ThreadUtils.awaitReady(waiter.completionFuture, Duration.Inf) waiter.completionFuture.value.get match { case scala.util.Success(_) => @@ -959,13 +954,12 @@ private[spark] class DAGScheduler( partitions: Array[Int], callSite: CallSite, listener: JobListener, - properties: Properties, - taskOverwritable: Boolean) { + properties: Properties) { var finalStage: ResultStage = null try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite, taskOverwritable) + finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite) } catch { case e: BarrierJobSlotsNumberCheckFailed => logWarning(s"The job $jobId requires to run a barrier stage that requires more slots " + @@ -979,7 +973,7 @@ private[spark] class DAGScheduler( messageScheduler.schedule( new Runnable { override def run(): Unit = eventProcessLoop.post(JobSubmitted(jobId, finalRDD, func, - partitions, callSite, listener, properties, taskOverwritable)) + partitions, callSite, listener, properties)) }, timeIntervalNumTasksCheck, TimeUnit.SECONDS @@ -1513,19 +1507,16 @@ private[spark] class DAGScheduler( failedStages += mapStage } - case resultStage: ResultStage => + case resultStage => val numMissingPartitions = resultStage.findMissingPartitions().length if (numMissingPartitions < resultStage.numTasks) { - if (resultStage.taskOverwritable) { - resultStage.activeJob.foreach(_.resetAllPartitions()) - } else { - val errorMessage = "A shuffle map stage with random output was failed " + - "and retried. However, Spark cannot rollback the result stage " + - s"$resultStage to re-process the input data, and has to fail this job. " + - "Please eliminate the randomness by checkpointing the RDD before " + - "repartition/zip and try again." - abortStage(failedStage, errorMessage, None) - } + // TODO: support to rollback result tasks. + val errorMessage = "A shuffle map stage with random output was failed and " + + s"retried. However, Spark cannot rollback the result stage $resultStage " + + "to re-process the input data, and has to fail this job. Please " + + "eliminate the randomness by checkpointing the RDD before " + + "repartition/zip and try again." + abortStage(failedStage, errorMessage, None) } } @@ -2052,16 +2043,8 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler } private def doOnReceive(event: DAGSchedulerEvent): Unit = event match { - case j: JobSubmitted => - dagScheduler.handleJobSubmitted( - j.jobId, - j.finalRDD, - j.func, - j.partitions, - j.callSite, - j.listener, - j.properties, - j.taskOverwritable) + case JobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) => + dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) case MapStageSubmitted(jobId, dependency, callSite, listener, properties) => dagScheduler.handleMapStageSubmitted(jobId, dependency, callSite, listener, properties) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 018def04ca16a..54ab8f8b3e1d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -41,8 +41,7 @@ private[scheduler] case class JobSubmitted( partitions: Array[Int], callSite: CallSite, listener: JobListener, - properties: Properties = null, - taskOverwritable: Boolean = false) + properties: Properties = null) extends DAGSchedulerEvent /** A map stage as submitted to run as a separate job */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala index ef4971eff3db7..d1687830ff7bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -34,8 +34,7 @@ private[spark] class ResultStage( val partitions: Array[Int], parents: List[Stage], firstJobId: Int, - callSite: CallSite, - val taskOverwritable: Boolean) + callSite: CallSite) extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite) { /** From 92179b8a841ec46457c8e8b1cf6f4b433753b57b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Aug 2018 08:38:17 +0800 Subject: [PATCH 09/26] rename --- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 4 ++-- .../src/main/scala/org/apache/spark/rdd/RDD.scala | 15 ++++++++------- .../apache/spark/rdd/ZippedPartitionsRDD.scala | 4 ++-- .../org/apache/spark/scheduler/DAGScheduler.scala | 6 +++--- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- 5 files changed, 16 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 42bfd23d58953..57c661bcb6fbc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -60,8 +60,8 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( isFromBarrier || dependencies.exists(_.rdd.isBarrier()) override private[spark] def computingRandomLevel = { - if (orderSensitiveFunc && prev.computingRandomLevel == RDD.RandomLevel.RANDOM_ORDER) { - RDD.RandomLevel.COMPLETE_RANDOM + if (orderSensitiveFunc && prev.computingRandomLevel == RDD.RandomLevel.UNORDERED) { + RDD.RandomLevel.INDETERMINATE } else { super.computingRandomLevel } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6ef101730b5df..971e8cd944c52 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1890,13 +1890,13 @@ abstract class RDD[T: ClassTag]( private[spark] def computingRandomLevel: RDD.RandomLevel.Value = { val parentRandomLevels = dependencies.map { case dep: ShuffleDependency[_, _, _] => - if (dep.rdd.computingRandomLevel == RDD.RandomLevel.COMPLETE_RANDOM) { - RDD.RandomLevel.COMPLETE_RANDOM + if (dep.rdd.computingRandomLevel == RDD.RandomLevel.INDETERMINATE) { + RDD.RandomLevel.INDETERMINATE } else { // In Spark, the reducer fetches multiple remote shuffle blocks at the same time, and // the arrival order of these shuffle blocks are totally random. Which means, the // computing function of a shuffled RDD will never be "NO_RANDOM" - RDD.RandomLevel.RANDOM_ORDER + RDD.RandomLevel.UNORDERED } case dep => dep.rdd.computingRandomLevel @@ -1923,16 +1923,17 @@ object RDD { /** * The random level of RDD's computing function, which indicates the behavior when rerun the * computing function. There are 3 random levels, ordered by the randomness from low to high: - * 1. IDEMPOTENT. The computing function always return the same result with same order when rerun. - * 2. RANDOM_ORDER. The computing function returns same data set in random order when rerun. - * 3. COMPLETE_RANDOM. The computing function returns different result when rerun. + * 1. IDEMPOTENT: The computing function always return the same result with same order when rerun. + * 2. UNORDERED: The computing function returns same data set in potentially a different order + * when rerun. + * 3. INDETERMINATE. The computing function may return totally different result when rerun. * * Note that, the output of the computing function usually relies on parent RDDs. When a * parent RDD's computing function is random, it's very likely this computing function is also * random. */ object RandomLevel extends Enumeration { - val IDEMPOTENT, RANDOM_ORDER, COMPLETE_RANDOM = Value + val IDEMPOTENT, UNORDERED, INDETERMINATE = Value } private[spark] val CHECKPOINT_ALL_MARKED_ANCESTORS = diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 2d0a20e79d636..183de9de8a8f7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -101,12 +101,12 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] } private def isRandomOrder(rdd: RDD[_]): Boolean = { - rdd.computingRandomLevel == RDD.RandomLevel.RANDOM_ORDER + rdd.computingRandomLevel == RDD.RandomLevel.UNORDERED } override private[spark] def computingRandomLevel = { if (orderSensitiveFunc && (isRandomOrder(rdd1) || isRandomOrder(rdd2))) { - RDD.RandomLevel.COMPLETE_RANDOM + RDD.RandomLevel.INDETERMINATE } else { super.computingRandomLevel } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 605bf2a440b12..93396ed4eb11e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1487,14 +1487,14 @@ private[spark] class DAGScheduler( failedStages += failedStage failedStages += mapStage if (noResubmitEnqueued) { - // If the map stage is COMPLETE_RANDOM, which means the map tasks will return + // If the map stage is INDETERMINATE, which means the map tasks may return // different result when re-try, we need to re-try all the tasks of the failed // stage and its succeeding stages, because the input data will be changed after the // map tasks are re-tried. - // Note that, if map stage is RANDOM_ORDER, we are fine. The shuffle partitioner is + // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is // guaranteed to be idempotent, so the input data of the reducers will not change even // if the map tasks are not re-tried. - if (mapStage.rdd.computingRandomLevel == RDD.RandomLevel.COMPLETE_RANDOM) { + if (mapStage.rdd.computingRandomLevel == RDD.RandomLevel.INDETERMINATE) { def rollBackStage(stage: Stage): Unit = stage match { case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 0adad0494271e..d3da208eafaca 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -82,7 +82,7 @@ class MyRDD( }).toArray override private[spark] def computingRandomLevel = { - if (isRandom) RDD.RandomLevel.COMPLETE_RANDOM else super.computingRandomLevel + if (isRandom) RDD.RandomLevel.INDETERMINATE else super.computingRandomLevel } override def getPreferredLocations(partition: Partition): Seq[String] = { From b564f4e1df5727ba1060c3883fb81fd7e77eddf9 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Aug 2018 08:43:21 +0800 Subject: [PATCH 10/26] fix a typo --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 93396ed4eb11e..56d84ceabeb9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1493,7 +1493,7 @@ private[spark] class DAGScheduler( // map tasks are re-tried. // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is // guaranteed to be idempotent, so the input data of the reducers will not change even - // if the map tasks are not re-tried. + // if the map tasks are re-tried. if (mapStage.rdd.computingRandomLevel == RDD.RandomLevel.INDETERMINATE) { def rollBackStage(stage: Stage): Unit = stage match { case mapStage: ShuffleMapStage => @@ -1502,7 +1502,7 @@ private[spark] class DAGScheduler( markStageAsFinished( mapStage, Some("preceding shuffle map stage with random output gets retried."), - willRetry = !shouldAbortStage) + willRetry = true) mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) failedStages += mapStage } From 23ce9a31bbbc015fff1b691a51b2d272c22369cf Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Aug 2018 17:13:28 +0800 Subject: [PATCH 11/26] address comments --- .../apache/spark/rdd/MapPartitionsRDD.scala | 10 +-- .../main/scala/org/apache/spark/rdd/RDD.scala | 87 ++++++++++--------- .../spark/rdd/ZippedPartitionsRDD.scala | 18 +--- .../apache/spark/scheduler/DAGScheduler.scala | 14 ++- .../spark/scheduler/DAGSchedulerSuite.scala | 10 +-- 5 files changed, 66 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 57c661bcb6fbc..99832cd8a013f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark.{Partition, TaskContext} * doesn't modify the keys. * @param isFromBarrier Indicates whether this RDD is transformed from an RDDBarrier, a stage * containing at least one RDDBarrier shall be turned into a barrier stage. - * @param orderSensitiveFunc whether or not the zip function is order-sensitive. If it's order + * @param orderSensitiveFunc whether or not the function is order-sensitive. If it's order * sensitive, it may return totally different result if the input order * changed. Mostly stateful functions are order-sensitive. */ @@ -59,11 +59,11 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( @transient protected lazy override val isBarrier_ : Boolean = isFromBarrier || dependencies.exists(_.rdd.isBarrier()) - override private[spark] def computingRandomLevel = { - if (orderSensitiveFunc && prev.computingRandomLevel == RDD.RandomLevel.UNORDERED) { - RDD.RandomLevel.INDETERMINATE + override private[spark] def outputRandomLevel = { + if (orderSensitiveFunc && prev.outputRandomLevel == RandomLevel.UNORDERED) { + RandomLevel.INDETERMINATE } else { - super.computingRandomLevel + super.outputRandomLevel } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 971e8cd944c52..6fc03e4097e47 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -808,8 +808,11 @@ abstract class RDD[T: ClassTag]( * serializable and don't require closure cleaning. * * @param preservesPartitioning indicates whether the input function preserves the partitioner, - * which should be `false` unless this is a pair RDD and the input function doesn't modify - * the keys. + * which should be `false` unless this is a pair RDD and the input + * function doesn't modify the keys. + * @param orderSensitiveFunc whether or not the function is order-sensitive. If it's order + * sensitive, it may return totally different result if the input order + * changed. Mostly stateful functions are order-sensitive. */ private[spark] def mapPartitionsWithIndexInternal[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], @@ -858,17 +861,16 @@ abstract class RDD[T: ClassTag]( * a map on the other). */ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope { - zipPartitionsInternal(other, preservesPartitioning = false, orderSensitiveFunc = true) { - (thisIter, otherIter) => - new Iterator[(T, U)] { - def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match { - case (true, true) => true - case (false, false) => false - case _ => throw new SparkException("Can only zip RDDs with " + - "same number of elements in each partition") - } - def next(): (T, U) = (thisIter.next(), otherIter.next()) + zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => + new Iterator[(T, U)] { + def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match { + case (true, true) => true + case (false, false) => false + case _ => throw new SparkException("Can only zip RDDs with " + + "same number of elements in each partition") } + def next(): (T, U) = (thisIter.next(), otherIter.next()) + } } } @@ -884,13 +886,6 @@ abstract class RDD[T: ClassTag]( new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning) } - private[spark] def zipPartitionsInternal[B: ClassTag, V: ClassTag] - (rdd2: RDD[B], preservesPartitioning: Boolean, orderSensitiveFunc: Boolean) - (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope { - new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning, orderSensitiveFunc) - } - - def zipPartitions[B: ClassTag, V: ClassTag] (rdd2: RDD[B]) (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope { @@ -1878,8 +1873,8 @@ abstract class RDD[T: ClassTag]( dependencies.filter(!_.isInstanceOf[ShuffleDependency[_, _, _]]).exists(_.rdd.isBarrier()) /** - * Returns the random level of this RDD's computing function. Please refer to [[RDD.RandomLevel]] - * for the definition of random level. + * Returns the random level of this RDD's output. Please refer to [[RandomLevel]] for the + * definition. * * By default, an RDD without parents(root RDD) is IDEMPOTENT. For RDDs with parents, the random * level of current RDD is the random level of the parent which is random most. @@ -1887,24 +1882,30 @@ abstract class RDD[T: ClassTag]( // TODO: make it public so users can set random level to their custom RDDs. // TODO: this can be per-partition. e.g. UnionRDD can have different random level for different // partitions. - private[spark] def computingRandomLevel: RDD.RandomLevel.Value = { + private[spark] def outputRandomLevel: RandomLevel.Value = { val parentRandomLevels = dependencies.map { case dep: ShuffleDependency[_, _, _] => - if (dep.rdd.computingRandomLevel == RDD.RandomLevel.INDETERMINATE) { - RDD.RandomLevel.INDETERMINATE + if (dep.rdd.outputRandomLevel == RandomLevel.IDEMPOTENT && + dep.keyOrdering.isDefined && dep.aggregator.isDefined) { + // if aggregator specified (and so unique keys) and key ordering specified - then + // consistent ordering. + RandomLevel.IDEMPOTENT + } else if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { + // If map output was indeterminate, shuffle output will be indeterminate as well + RandomLevel.INDETERMINATE } else { // In Spark, the reducer fetches multiple remote shuffle blocks at the same time, and // the arrival order of these shuffle blocks are totally random. Which means, the // computing function of a shuffled RDD will never be "NO_RANDOM" - RDD.RandomLevel.UNORDERED + RandomLevel.UNORDERED } - case dep => dep.rdd.computingRandomLevel + case dep => dep.rdd.outputRandomLevel } if (parentRandomLevels.isEmpty) { // By default we assume the root RDD is idempotent - RDD.RandomLevel.IDEMPOTENT + RandomLevel.IDEMPOTENT } else { parentRandomLevels.maxBy(_.id) } @@ -1920,22 +1921,6 @@ abstract class RDD[T: ClassTag]( */ object RDD { - /** - * The random level of RDD's computing function, which indicates the behavior when rerun the - * computing function. There are 3 random levels, ordered by the randomness from low to high: - * 1. IDEMPOTENT: The computing function always return the same result with same order when rerun. - * 2. UNORDERED: The computing function returns same data set in potentially a different order - * when rerun. - * 3. INDETERMINATE. The computing function may return totally different result when rerun. - * - * Note that, the output of the computing function usually relies on parent RDDs. When a - * parent RDD's computing function is random, it's very likely this computing function is also - * random. - */ - object RandomLevel extends Enumeration { - val IDEMPOTENT, UNORDERED, INDETERMINATE = Value - } - private[spark] val CHECKPOINT_ALL_MARKED_ANCESTORS = "spark.checkpoint.checkpointAllMarkedAncestors" @@ -1978,3 +1963,19 @@ object RDD { new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) } } + +/** + * The random level of RDD's output (i.e. what `RDD#compute` returns), which indicates how the + * output will diff when Spark reruns the tasks for the RDD. There are 3 random levels, ordered + * by the randomness from low to high: + * 1. IDEMPOTENT: The RDD output is always same (including order) when rerun. + * 2. UNORDERED: The RDD output is always the same data set but in potentially a different order + * when rerun. + * 3. INDETERMINATE. The RDD output can be different (not only order) when rerun. + * + * Note that, the output of an RDD usually relies on parent RDDs. When a parent RDD's output is + * INDETERMINATE, it's very likely this RDD's output is also INDETERMINATE. + */ +private[spark] object RandomLevel extends Enumeration { + val IDEMPOTENT, UNORDERED, INDETERMINATE = Value +} diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 183de9de8a8f7..3cb1231bd3477 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -81,11 +81,7 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] var f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], - preservesPartitioning: Boolean = false, - // whether or not the zip function is order-sensitive. If it's order sensitive, it may return - // totally different result if the input order changed. As an example, `iter1.zip(iter2)` is - // order sensitive. - orderSensitiveFunc: Boolean = false) + preservesPartitioning: Boolean = false) extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { @@ -99,18 +95,6 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] rdd2 = null f = null } - - private def isRandomOrder(rdd: RDD[_]): Boolean = { - rdd.computingRandomLevel == RDD.RandomLevel.UNORDERED - } - - override private[spark] def computingRandomLevel = { - if (orderSensitiveFunc && (isRandomOrder(rdd1) || isRandomOrder(rdd2))) { - RDD.RandomLevel.INDETERMINATE - } else { - super.computingRandomLevel - } - } } private[spark] class ZippedPartitionsRDD3 diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 56d84ceabeb9b..bcca4cb8ed804 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -40,7 +40,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.rdd.{RandomLevel, RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1494,7 +1494,7 @@ private[spark] class DAGScheduler( // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is // guaranteed to be idempotent, so the input data of the reducers will not change even // if the map tasks are re-tried. - if (mapStage.rdd.computingRandomLevel == RDD.RandomLevel.INDETERMINATE) { + if (mapStage.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { def rollBackStage(stage: Stage): Unit = stage match { case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length @@ -1526,10 +1526,18 @@ private[spark] class DAGScheduler( if (!failedStages.contains(stage)) rollBackStage(stage) } } else { - stageChain.head.parents.foreach(s => rollbackSucceedingStages(s :: stageChain)) + stageChain.head.parents.foreach { s => + // The stage may already be processed in another DAG, skip it if it's in + // `failedStages`. + if (!failedStages.contains(s)) { + rollbackSucceedingStages(s :: stageChain) + } + } } } + // `failedStage` is already added to `failedStages`, call `rollBackStage` here to + // rollback it, otherwise we will skip it in `rollbackSucceedingStages`. rollBackStage(failedStage) activeJobs.foreach(job => rollbackSucceedingStages(job.finalStage :: Nil)) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index d3da208eafaca..e89f11c8bba2f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,7 +30,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RandomLevel, RDD} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -71,7 +71,7 @@ class MyRDD( dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil, @(transient @param) tracker: MapOutputTrackerMaster = null, - isRandom: Boolean = false) + indeterminate: Boolean = false) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = @@ -81,8 +81,8 @@ class MyRDD( override def index: Int = i }).toArray - override private[spark] def computingRandomLevel = { - if (isRandom) RDD.RandomLevel.INDETERMINATE else super.computingRandomLevel + override private[spark] def outputRandomLevel = { + if (indeterminate) RandomLevel.INDETERMINATE else super.outputRandomLevel } override def getPreferredLocations(partition: Partition): Seq[String] = { @@ -2639,7 +2639,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("SPARK-23207: retry all the succeeding stages when the map stage is random") { - val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, isRandom = true) + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) val shuffleId1 = shuffleDep1.shuffleId From 33fe2895a2d38c0bd3eebaf412c4481503f20c11 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Aug 2018 17:23:20 +0800 Subject: [PATCH 12/26] more doc --- .../org/apache/spark/scheduler/DAGScheduler.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index bcca4cb8ed804..7e77533ddbde4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1520,18 +1520,19 @@ private[spark] class DAGScheduler( } } + // Rollback stages who is a descendant of the `failedStage`. def rollbackSucceedingStages(stageChain: List[Stage]): Unit = { if (stageChain.head.id == failedStage.id) { stageChain.foreach { stage => + // This stage may already be rollbacked with another stage chain, skip it if + // it's in `failedStages`. if (!failedStages.contains(stage)) rollBackStage(stage) } } else { stageChain.head.parents.foreach { s => - // The stage may already be processed in another DAG, skip it if it's in - // `failedStages`. - if (!failedStages.contains(s)) { - rollbackSucceedingStages(s :: stageChain) - } + // This stage may already be rollbacked with another stage chain, skip it if + // it's in `failedStages`. + if (!failedStages.contains(s)) rollbackSucceedingStages(s :: stageChain) } } } From 4e25b45ee172e74935cd817f29787d3691c9e7cc Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Aug 2018 20:41:23 +0800 Subject: [PATCH 13/26] one more test --- .../apache/spark/scheduler/DAGScheduler.scala | 7 ++--- .../spark/scheduler/DAGSchedulerSuite.scala | 31 +++++++++++++++++++ 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7e77533ddbde4..25ff4597969ca 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1512,10 +1512,9 @@ private[spark] class DAGScheduler( if (numMissingPartitions < resultStage.numTasks) { // TODO: support to rollback result tasks. val errorMessage = "A shuffle map stage with random output was failed and " + - s"retried. However, Spark cannot rollback the result stage $resultStage " + - "to re-process the input data, and has to fail this job. Please " + - "eliminate the randomness by checkpointing the RDD before " + - "repartition/zip and try again." + s"retried. However, Spark cannot rollback the $resultStage to re-process " + + "the input data, and has to fail this job. Please eliminate the " + + "randomness by checkpointing the RDD before repartition/zip and try again." abortStage(failedStage, errorMessage, None) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index e89f11c8bba2f..60c32cf8c4759 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2650,11 +2650,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) submit(finalRdd, Array(0, 1)) + // Finish the first shuffle map stage. complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + // Finish the second shuffle map stage. complete(taskSets(1), Seq( (Success, makeMapStatus("hostC", 2)), (Success, makeMapStatus("hostD", 2)))) @@ -2712,6 +2714,35 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() } + test("SPARK-23207: cannot rollback a result stage") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleId = shuffleDep.shuffleId + + val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) + submit(finalRdd, Array(0, 1)) + + // Finish the shuffle map stage. + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) + + // Finish the first task of the result stage + runEvent(makeCompletionEvent( + taskSets(1).tasks(0), Success, 42, + Seq.empty, createFakeTaskInfoWithId(0))) + + // Fail the second task with FetchFailed. + runEvent(makeCompletionEvent( + taskSets(1).tasks(1), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + null)) + + // The job should fail because Spark can't rollback the result stage. + assert(failure != null && failure.getMessage.contains("Spark cannot rollback")) + } + /** * 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. From 35ba626d1c4db1a27606f3a3b69f2a1621aeca6a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Aug 2018 21:41:16 +0800 Subject: [PATCH 14/26] code cleanup --- .../main/scala/org/apache/spark/rdd/RDD.scala | 25 ++++----- .../apache/spark/scheduler/DAGScheduler.scala | 53 +++++++++---------- 2 files changed, 39 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6fc03e4097e47..e9476ed278edc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1876,38 +1876,39 @@ abstract class RDD[T: ClassTag]( * Returns the random level of this RDD's output. Please refer to [[RandomLevel]] for the * definition. * - * By default, an RDD without parents(root RDD) is IDEMPOTENT. For RDDs with parents, the random - * level of current RDD is the random level of the parent which is random most. + * By default, an RDD without parents(root RDD) is IDEMPOTENT. For RDDs with parents, we will + * generate a random level candidate per parent according to the dependency. The random level + * of the current RDD is the random level candidate that is random most. */ // TODO: make it public so users can set random level to their custom RDDs. // TODO: this can be per-partition. e.g. UnionRDD can have different random level for different // partitions. private[spark] def outputRandomLevel: RandomLevel.Value = { - val parentRandomLevels = dependencies.map { + val randomLevelCandidates = dependencies.map { case dep: ShuffleDependency[_, _, _] => - if (dep.rdd.outputRandomLevel == RandomLevel.IDEMPOTENT && - dep.keyOrdering.isDefined && dep.aggregator.isDefined) { + if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { + // If map output was indeterminate, shuffle output will be indeterminate as well + RandomLevel.INDETERMINATE + } else if (dep.keyOrdering.isDefined && dep.aggregator.isDefined) { // if aggregator specified (and so unique keys) and key ordering specified - then // consistent ordering. RandomLevel.IDEMPOTENT - } else if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { - // If map output was indeterminate, shuffle output will be indeterminate as well - RandomLevel.INDETERMINATE } else { // In Spark, the reducer fetches multiple remote shuffle blocks at the same time, and - // the arrival order of these shuffle blocks are totally random. Which means, the - // computing function of a shuffled RDD will never be "NO_RANDOM" + // the arrival order of these shuffle blocks are totally random. Even if the parent map + // RDD is IDEMPOTENT, the reduce RDD is always UNORDERED. RandomLevel.UNORDERED } + // For narrow dependency, assume the output random level of current RDD is same as parent. case dep => dep.rdd.outputRandomLevel } - if (parentRandomLevels.isEmpty) { + if (randomLevelCandidates.isEmpty) { // By default we assume the root RDD is idempotent RandomLevel.IDEMPOTENT } else { - parentRandomLevels.maxBy(_.id) + randomLevelCandidates.maxBy(_.id) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 25ff4597969ca..2122fa4cc9c1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1495,7 +1495,27 @@ private[spark] class DAGScheduler( // guaranteed to be idempotent, so the input data of the reducers will not change even // if the map tasks are re-tried. if (mapStage.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { - def rollBackStage(stage: Stage): Unit = stage match { + // It's a little tricky to find all the succeeding stages of `failedStage`, because + // each stage only know its parents not children. Here we traverse the stages from + // the leaf nodes (the result stages of active jobs), and rollback all the stages + // in the stage chains that connect to the `failedStage`. To speed up the stage + // traversing, we collect the stages to rollback first. If a stage needs to + // rollback, all its succeeding stages need to rollback to. + val stagesToRollback = scala.collection.mutable.HashSet(failedStage) + + def collectStagesToRollback(stageChain: List[Stage]): Unit = { + if (stagesToRollback.contains(stageChain.head)) { + stageChain.drop(1).foreach(s => stagesToRollback += s) + } else { + stageChain.head.parents.foreach { s => + collectStagesToRollback(s :: stageChain) + } + } + } + + activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) + + stagesToRollback.foreach { case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { @@ -1511,35 +1531,14 @@ private[spark] class DAGScheduler( val numMissingPartitions = resultStage.findMissingPartitions().length if (numMissingPartitions < resultStage.numTasks) { // TODO: support to rollback result tasks. - val errorMessage = "A shuffle map stage with random output was failed and " + - s"retried. However, Spark cannot rollback the $resultStage to re-process " + - "the input data, and has to fail this job. Please eliminate the " + - "randomness by checkpointing the RDD before repartition/zip and try again." + val errorMessage = "A shuffle map stage with random output was failed " + + s"and retried. However, Spark cannot rollback the $resultStage to " + + "re-process the input data, and has to fail this job. Please " + + "eliminate the randomness by checkpointing the RDD before " + + "repartition/zip and try again." abortStage(failedStage, errorMessage, None) } } - - // Rollback stages who is a descendant of the `failedStage`. - def rollbackSucceedingStages(stageChain: List[Stage]): Unit = { - if (stageChain.head.id == failedStage.id) { - stageChain.foreach { stage => - // This stage may already be rollbacked with another stage chain, skip it if - // it's in `failedStages`. - if (!failedStages.contains(stage)) rollBackStage(stage) - } - } else { - stageChain.head.parents.foreach { s => - // This stage may already be rollbacked with another stage chain, skip it if - // it's in `failedStages`. - if (!failedStages.contains(s)) rollbackSucceedingStages(s :: stageChain) - } - } - } - - // `failedStage` is already added to `failedStages`, call `rollBackStage` here to - // rollback it, otherwise we will skip it in `rollbackSucceedingStages`. - rollBackStage(failedStage) - activeJobs.foreach(job => rollbackSucceedingStages(job.finalStage :: Nil)) } // We expect one executor failure to trigger many FetchFailures in rapid succession, From dbdc21c4276ae8fd18f7c58f53168c26e6483810 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Aug 2018 22:14:14 +0800 Subject: [PATCH 15/26] fix checkpoint --- .../org/apache/spark/rdd/LocalCheckpointRDD.scala | 10 ++++++++-- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 5 +++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala index 503aa0dffc9f3..d9e53e7d29d8a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala @@ -37,11 +37,12 @@ import org.apache.spark.storage.RDDBlockId private[spark] class LocalCheckpointRDD[T: ClassTag]( sc: SparkContext, rddId: Int, - numPartitions: Int) + numPartitions: Int, + randomLevel: RandomLevel.Value) extends CheckpointRDD[T](sc) { def this(rdd: RDD[T]) { - this(rdd.context, rdd.id, rdd.partitions.length) + this(rdd.context, rdd.id, rdd.partitions.length, rdd.outputRandomLevel) } protected override def getPartitions: Array[Partition] = { @@ -64,4 +65,9 @@ private[spark] class LocalCheckpointRDD[T: ClassTag]( s"instead, which is slower than local checkpointing but more fault-tolerant.") } + // Local checkpoint is not reliable, we may still get output from original RDD, so the output + // random level should also inherent from the original RDD. + override private[spark] def outputRandomLevel = { + randomLevel + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e9476ed278edc..3efd3b650d5b0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1885,6 +1885,11 @@ abstract class RDD[T: ClassTag]( // partitions. private[spark] def outputRandomLevel: RandomLevel.Value = { val randomLevelCandidates = dependencies.map { + // If checkpointed to reliable store, then it's idempotent, as `ReliableCheckpointRDD` has + // same output (including data order) when rerun. + case dep: Dependency[_] if dep.rdd.getCheckpointFile.isDefined => + RandomLevel.IDEMPOTENT + case dep: ShuffleDependency[_, _, _] => if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { // If map output was indeterminate, shuffle output will be indeterminate as well From 91f057dd26f40915d7c014b3fd6f8f333fe85f15 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Aug 2018 22:58:04 +0800 Subject: [PATCH 16/26] address comment --- .../org/apache/spark/scheduler/DAGScheduler.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2122fa4cc9c1b..8369819a04ea3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1519,10 +1519,15 @@ private[spark] class DAGScheduler( case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - markStageAsFinished( - mapStage, - Some("preceding shuffle map stage with random output gets retried."), - willRetry = true) + val reason = "preceding shuffle map stage with random output gets retried." + try { + taskScheduler.killAllTaskAttempts( + mapStage.id, interruptThread = false, reason) + } catch { + case e: UnsupportedOperationException => + logWarning(s"Could not kill all tasks for $mapStage", e) + } + markStageAsFinished(mapStage, Some(reason), willRetry = true) mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) failedStages += mapStage } From dd7474b1130dd3e317fb10eca75c56e512c60dff Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 25 Aug 2018 01:28:21 +0800 Subject: [PATCH 17/26] add tests and fix checkpoint --- .../apache/spark/rdd/LocalCheckpointRDD.scala | 2 +- .../apache/spark/rdd/MapPartitionsRDD.scala | 4 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 24 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 103 +++++++++++++++--- 4 files changed, 104 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala index d9e53e7d29d8a..dfb7fbe6b41e7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala @@ -67,7 +67,7 @@ private[spark] class LocalCheckpointRDD[T: ClassTag]( // Local checkpoint is not reliable, we may still get output from original RDD, so the output // random level should also inherent from the original RDD. - override private[spark] def outputRandomLevel = { + override protected def getOutputRandomLevel = { randomLevel } } diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 99832cd8a013f..44b945bdd64ba 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -59,11 +59,11 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( @transient protected lazy override val isBarrier_ : Boolean = isFromBarrier || dependencies.exists(_.rdd.isBarrier()) - override private[spark] def outputRandomLevel = { + override protected def getOutputRandomLevel = { if (orderSensitiveFunc && prev.outputRandomLevel == RandomLevel.UNORDERED) { RandomLevel.INDETERMINATE } else { - super.outputRandomLevel + super.getOutputRandomLevel } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3efd3b650d5b0..5ab32f2014ef4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1876,20 +1876,26 @@ abstract class RDD[T: ClassTag]( * Returns the random level of this RDD's output. Please refer to [[RandomLevel]] for the * definition. * - * By default, an RDD without parents(root RDD) is IDEMPOTENT. For RDDs with parents, we will - * generate a random level candidate per parent according to the dependency. The random level - * of the current RDD is the random level candidate that is random most. + * By default, an reliably checkpointed RDD, or RDD without parents(root RDD) is IDEMPOTENT. For + * RDDs with parents, we will generate a random level candidate per parent according to the + * dependency. The random level of the current RDD is the random level candidate that is random + * most. Please override [[getOutputRandomLevel]] to provide custom logic of calculating output + * random level. */ // TODO: make it public so users can set random level to their custom RDDs. // TODO: this can be per-partition. e.g. UnionRDD can have different random level for different // partitions. - private[spark] def outputRandomLevel: RandomLevel.Value = { - val randomLevelCandidates = dependencies.map { - // If checkpointed to reliable store, then it's idempotent, as `ReliableCheckpointRDD` has - // same output (including data order) when rerun. - case dep: Dependency[_] if dep.rdd.getCheckpointFile.isDefined => - RandomLevel.IDEMPOTENT + private[spark] final lazy val outputRandomLevel: RandomLevel.Value = { + if (checkpointData.exists(_.isInstanceOf[ReliableRDDCheckpointData[_]])) { + RandomLevel.IDEMPOTENT + } else { + getOutputRandomLevel + } + } + @DeveloperApi + protected def getOutputRandomLevel: RandomLevel.Value = { + val randomLevelCandidates = dependencies.map { case dep: ShuffleDependency[_, _, _] => if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { // If map output was indeterminate, shuffle output will be indeterminate as well diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 60c32cf8c4759..ab0c9fc9b7be8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -81,8 +81,8 @@ class MyRDD( override def index: Int = i }).toArray - override private[spark] def outputRandomLevel = { - if (indeterminate) RandomLevel.INDETERMINATE else super.outputRandomLevel + override protected def getOutputRandomLevel = { + if (indeterminate) RandomLevel.INDETERMINATE else super.getOutputRandomLevel } override def getPreferredLocations(partition: Partition): Seq[String] = { @@ -2640,14 +2640,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi test("SPARK-23207: retry all the succeeding stages when the map stage is random") { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) val shuffleId1 = shuffleDep1.shuffleId - val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) val shuffleId2 = shuffleDep2.shuffleId - val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) + submit(finalRdd, Array(0, 1)) // Finish the first shuffle map stage. @@ -2675,10 +2676,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0)) // The result stage is still waiting for its 2 tasks to complete - val resultStage = failedStages.collect { + assert(failedStages.collect { case stage: ResultStage => stage - }.head - assert(resultStage.findMissingPartitions() == Seq(0, 1)) + }.head.findMissingPartitions() == Seq(0, 1)) scheduler.resubmitFailedStages() @@ -2714,28 +2714,24 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() } - test("SPARK-23207: cannot rollback a result stage") { - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) - val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = { + val shuffleDep = new ShuffleDependency(mapRdd, new HashPartitioner(2)) val shuffleId = shuffleDep.shuffleId - val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) + submit(finalRdd, Array(0, 1)) - // Finish the shuffle map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(taskSets.length - 1, 0, numShufflePartitions = 2) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) // Finish the first task of the result stage runEvent(makeCompletionEvent( - taskSets(1).tasks(0), Success, 42, + taskSets.last.tasks(0), Success, 42, Seq.empty, createFakeTaskInfoWithId(0))) // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( - taskSets(1).tasks(1), + taskSets.last.tasks(1), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) @@ -2743,6 +2739,79 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(failure != null && failure.getMessage.contains("Spark cannot rollback")) } + test("SPARK-23207: cannot rollback a result stage") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + assertResultStageFailToRollback(shuffleMapRdd) + } + + test("SPARK-23207: local checkpoint fail to rollback (checkpointed before)") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.localCheckpoint() + submit(shuffleMapRdd, Array(1, 1)) + completeNextResultStageWithSuccess(0, 0) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assertDataStructuresEmpty() + assertResultStageFailToRollback(shuffleMapRdd) + } + + test("SPARK-23207: local checkpoint fail to rollback (checkpointing now)") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.localCheckpoint() + assertResultStageFailToRollback(shuffleMapRdd) + } + + private def assertResultStageNotRollbacked(mapRdd: MyRDD): Unit = { + val shuffleDep = new ShuffleDependency(mapRdd, new HashPartitioner(2)) + val shuffleId = shuffleDep.shuffleId + val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) + + submit(finalRdd, Array(0, 1)) + + completeShuffleMapStageSuccessfully(taskSets.length - 1, 0, numShufflePartitions = 2) + assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) + + // Finish the first task of the result stage + runEvent(makeCompletionEvent( + taskSets.last.tasks(0), Success, 42, + Seq.empty, createFakeTaskInfoWithId(0))) + + // Fail the second task with FetchFailed. + runEvent(makeCompletionEvent( + taskSets.last.tasks(1), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + null)) + + assert(failure == null, "job should not fail") + val failedStages = scheduler.failedStages.toSeq + assert(failedStages.length == 2) + // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd2` needs to retry. + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId => stage + }.head.findMissingPartitions() == Seq(0)) + // The first task of result stage remains completed. + assert(failedStages.collect { + case stage: ResultStage => stage + }.head.findMissingPartitions() == Seq(1)) + } + + test("SPARK-23207: reliable checkpoint can avoid rollback (checkpointed before)") { + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.checkpoint() + submit(shuffleMapRdd, Array(1, 1)) + completeNextResultStageWithSuccess(0, 0) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assertDataStructuresEmpty() + assertResultStageNotRollbacked(shuffleMapRdd) + } + + test("SPARK-23207: reliable checkpoint can avoid rollback (checkpointing now)") { + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.checkpoint() + assertResultStageNotRollbacked(shuffleMapRdd) + } + /** * 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. From cba588d5642b043aa807cfe675ad4f7f6adef17c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 25 Aug 2018 11:51:57 +0800 Subject: [PATCH 18/26] address comment --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5ab32f2014ef4..f2aae09963f99 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1896,6 +1896,11 @@ abstract class RDD[T: ClassTag]( @DeveloperApi protected def getOutputRandomLevel: RandomLevel.Value = { val randomLevelCandidates = dependencies.map { + // The shuffle is not really happening, treat it like narrow dependency and assume the output + // random level of current RDD is same as parent. + case dep: ShuffleDependency[_, _, _] if dep.rdd.partitioner.exists(_ == dep.partitioner) => + dep.rdd.outputRandomLevel + case dep: ShuffleDependency[_, _, _] => if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { // If map output was indeterminate, shuffle output will be indeterminate as well From 77440e774e55eca1b56992ce2ec23f2ad666cab0 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 28 Aug 2018 14:20:26 +0800 Subject: [PATCH 19/26] update --- .../scala/org/apache/spark/Partitioner.scala | 4 +- .../apache/spark/rdd/LocalCheckpointRDD.scala | 10 +-- .../apache/spark/rdd/MapPartitionsRDD.scala | 8 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 81 +++++++++++-------- .../apache/spark/scheduler/DAGScheduler.scala | 19 ++--- .../spark/scheduler/DAGSchedulerSuite.scala | 10 +-- 6 files changed, 72 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index fdb5df21ac9ff..515237558fd87 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -34,8 +34,8 @@ import org.apache.spark.util.random.SamplingUtils * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. * - * Note that, partitioner must be idempotent, i.e. it must return the same partition id given the - * same partition key. + * Note that, partitioner must be deterministic, i.e. it must return the same partition id given + * the same partition key. */ abstract class Partitioner extends Serializable { def numPartitions: Int diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala index dfb7fbe6b41e7..cd04737b816fa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala @@ -38,11 +38,11 @@ private[spark] class LocalCheckpointRDD[T: ClassTag]( sc: SparkContext, rddId: Int, numPartitions: Int, - randomLevel: RandomLevel.Value) + deterministicLevel: DeterministicLevel.Value) extends CheckpointRDD[T](sc) { def this(rdd: RDD[T]) { - this(rdd.context, rdd.id, rdd.partitions.length, rdd.outputRandomLevel) + this(rdd.context, rdd.id, rdd.partitions.length, rdd.outputDeterministicLevel) } protected override def getPartitions: Array[Partition] = { @@ -66,8 +66,8 @@ private[spark] class LocalCheckpointRDD[T: ClassTag]( } // Local checkpoint is not reliable, we may still get output from original RDD, so the output - // random level should also inherent from the original RDD. - override protected def getOutputRandomLevel = { - randomLevel + // deterministic level should also inherent from the original RDD. + override protected def getOutputDeterministicLevel = { + deterministicLevel } } diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 44b945bdd64ba..64bd7199c2392 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -59,11 +59,11 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( @transient protected lazy override val isBarrier_ : Boolean = isFromBarrier || dependencies.exists(_.rdd.isBarrier()) - override protected def getOutputRandomLevel = { - if (orderSensitiveFunc && prev.outputRandomLevel == RandomLevel.UNORDERED) { - RandomLevel.INDETERMINATE + override protected def getOutputDeterministicLevel = { + if (orderSensitiveFunc && prev.outputDeterministicLevel == DeterministicLevel.UNORDERED) { + DeterministicLevel.INDETERMINATE } else { - super.getOutputRandomLevel + super.getOutputDeterministicLevel } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index f2aae09963f99..73f15accd2b6f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1642,6 +1642,16 @@ abstract class RDD[T: ClassTag]( } } + /** + * Return whether this RDD is reliably checkpointed and materialized. + */ + private[rdd] def isReliablyCheckpointed: Boolean = { + checkpointData match { + case Some(reliable: ReliableRDDCheckpointData[_]) if reliable.isCheckpointed => true + case _ => false + } + } + /** * Gets the name of the directory to which this RDD was checkpointed. * This is not defined if the RDD is checkpointed locally. @@ -1873,58 +1883,59 @@ abstract class RDD[T: ClassTag]( dependencies.filter(!_.isInstanceOf[ShuffleDependency[_, _, _]]).exists(_.rdd.isBarrier()) /** - * Returns the random level of this RDD's output. Please refer to [[RandomLevel]] for the - * definition. + * Returns the deterministic level of this RDD's output. Please refer to [[DeterministicLevel]] + * for the definition. * - * By default, an reliably checkpointed RDD, or RDD without parents(root RDD) is IDEMPOTENT. For - * RDDs with parents, we will generate a random level candidate per parent according to the - * dependency. The random level of the current RDD is the random level candidate that is random - * most. Please override [[getOutputRandomLevel]] to provide custom logic of calculating output - * random level. + * By default, an reliably checkpointed RDD, or RDD without parents(root RDD) is DETERMINATE. For + * RDDs with parents, we will generate a deterministic level candidate per parent according to + * the dependency. The deterministic level of the current RDD is the deterministic level + * candidate that is deterministic least. Please override [[getOutputDeterministicLevel]] to + * provide custom logic of calculating output deterministic level. */ - // TODO: make it public so users can set random level to their custom RDDs. - // TODO: this can be per-partition. e.g. UnionRDD can have different random level for different - // partitions. - private[spark] final lazy val outputRandomLevel: RandomLevel.Value = { - if (checkpointData.exists(_.isInstanceOf[ReliableRDDCheckpointData[_]])) { - RandomLevel.IDEMPOTENT + // TODO: make it public so users can set deterministic level to their custom RDDs. + // TODO: this can be per-partition. e.g. UnionRDD can have different deterministic level for + // different partitions. + private[spark] final lazy val outputDeterministicLevel: DeterministicLevel.Value = { + if (isReliablyCheckpointed) { + DeterministicLevel.DETERMINATE } else { - getOutputRandomLevel + getOutputDeterministicLevel } } @DeveloperApi - protected def getOutputRandomLevel: RandomLevel.Value = { - val randomLevelCandidates = dependencies.map { + protected def getOutputDeterministicLevel: DeterministicLevel.Value = { + val deterministicLevelCandidates = dependencies.map { // The shuffle is not really happening, treat it like narrow dependency and assume the output - // random level of current RDD is same as parent. + // deterministic level of current RDD is same as parent. case dep: ShuffleDependency[_, _, _] if dep.rdd.partitioner.exists(_ == dep.partitioner) => - dep.rdd.outputRandomLevel + dep.rdd.outputDeterministicLevel case dep: ShuffleDependency[_, _, _] => - if (dep.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { + if (dep.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) { // If map output was indeterminate, shuffle output will be indeterminate as well - RandomLevel.INDETERMINATE + DeterministicLevel.INDETERMINATE } else if (dep.keyOrdering.isDefined && dep.aggregator.isDefined) { // if aggregator specified (and so unique keys) and key ordering specified - then // consistent ordering. - RandomLevel.IDEMPOTENT + DeterministicLevel.DETERMINATE } else { // In Spark, the reducer fetches multiple remote shuffle blocks at the same time, and // the arrival order of these shuffle blocks are totally random. Even if the parent map - // RDD is IDEMPOTENT, the reduce RDD is always UNORDERED. - RandomLevel.UNORDERED + // RDD is DETERMINATE, the reduce RDD is always UNORDERED. + DeterministicLevel.UNORDERED } - // For narrow dependency, assume the output random level of current RDD is same as parent. - case dep => dep.rdd.outputRandomLevel + // For narrow dependency, assume the output deterministic level of current RDD is same as + // parent. + case dep => dep.rdd.outputDeterministicLevel } - if (randomLevelCandidates.isEmpty) { - // By default we assume the root RDD is idempotent - RandomLevel.IDEMPOTENT + if (deterministicLevelCandidates.isEmpty) { + // By default we assume the root RDD is determinate. + DeterministicLevel.DETERMINATE } else { - randomLevelCandidates.maxBy(_.id) + deterministicLevelCandidates.maxBy(_.id) } } } @@ -1982,10 +1993,10 @@ object RDD { } /** - * The random level of RDD's output (i.e. what `RDD#compute` returns), which indicates how the - * output will diff when Spark reruns the tasks for the RDD. There are 3 random levels, ordered - * by the randomness from low to high: - * 1. IDEMPOTENT: The RDD output is always same (including order) when rerun. + * The deterministic level of RDD's output (i.e. what `RDD#compute` returns), which indicates how + * the output will diff when Spark reruns the tasks for the RDD. There are 3 deterministic levels, + * ordered by the determinism from high to low: + * 1. DETERMINATE: The RDD output is always same (including order) when rerun. * 2. UNORDERED: The RDD output is always the same data set but in potentially a different order * when rerun. * 3. INDETERMINATE. The RDD output can be different (not only order) when rerun. @@ -1993,6 +2004,6 @@ object RDD { * Note that, the output of an RDD usually relies on parent RDDs. When a parent RDD's output is * INDETERMINATE, it's very likely this RDD's output is also INDETERMINATE. */ -private[spark] object RandomLevel extends Enumeration { - val IDEMPOTENT, UNORDERED, INDETERMINATE = Value +private[spark] object DeterministicLevel extends Enumeration { + val DETERMINATE, UNORDERED, INDETERMINATE = Value } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8369819a04ea3..4001bafd3d816 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -40,7 +40,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{RandomLevel, RDD, RDDCheckpointData} +import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1492,9 +1492,9 @@ private[spark] class DAGScheduler( // stage and its succeeding stages, because the input data will be changed after the // map tasks are re-tried. // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is - // guaranteed to be idempotent, so the input data of the reducers will not change even - // if the map tasks are re-tried. - if (mapStage.rdd.outputRandomLevel == RandomLevel.INDETERMINATE) { + // guaranteed to be determinate, so the input data of the reducers will not change + // even if the map tasks are re-tried. + if (mapStage.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) { // It's a little tricky to find all the succeeding stages of `failedStage`, because // each stage only know its parents not children. Here we traverse the stages from // the leaf nodes (the result stages of active jobs), and rollback all the stages @@ -1519,7 +1519,8 @@ private[spark] class DAGScheduler( case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - val reason = "preceding shuffle map stage with random output gets retried." + val reason = "preceding shuffle map stage with indeterminate output gets " + + "retried." try { taskScheduler.killAllTaskAttempts( mapStage.id, interruptThread = false, reason) @@ -1536,10 +1537,10 @@ private[spark] class DAGScheduler( val numMissingPartitions = resultStage.findMissingPartitions().length if (numMissingPartitions < resultStage.numTasks) { // TODO: support to rollback result tasks. - val errorMessage = "A shuffle map stage with random output was failed " + - s"and retried. However, Spark cannot rollback the $resultStage to " + - "re-process the input data, and has to fail this job. Please " + - "eliminate the randomness by checkpointing the RDD before " + + val errorMessage = "A shuffle map stage with indeterminate output was " + + s"failed and retried. However, Spark cannot rollback the $resultStage " + + "to re-process the input data, and has to fail this job. Please " + + "eliminate the determinism by checkpointing the RDD before " + "repartition/zip and try again." abortStage(failedStage, errorMessage, None) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ab0c9fc9b7be8..8d4f72190fc4d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,7 +30,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager -import org.apache.spark.rdd.{RandomLevel, RDD} +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -81,8 +81,8 @@ class MyRDD( override def index: Int = i }).toArray - override protected def getOutputRandomLevel = { - if (indeterminate) RandomLevel.INDETERMINATE else super.getOutputRandomLevel + override protected def getOutputDeterministicLevel = { + if (indeterminate) DeterministicLevel.INDETERMINATE else super.getOutputDeterministicLevel } override def getPreferredLocations(partition: Partition): Seq[String] = { @@ -2638,7 +2638,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 2) } - test("SPARK-23207: retry all the succeeding stages when the map stage is random") { + test("SPARK-23207: retry all the succeeding stages when the map stage is indeterminate") { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) @@ -2694,7 +2694,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId1 => stage }.head.findMissingPartitions() == Seq(0)) - // The second shuffle map stage is entirely rollbacked, because the root RDD is random. + // The second shuffle map stage is entirely rollbacked, because the root RDD is indeterminate. assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0, 1)) From 5371d21571153169cfc9b8a1cb2d600a737b6d78 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 28 Aug 2018 23:50:27 +0800 Subject: [PATCH 20/26] update tests --- .../spark/scheduler/DAGSchedulerSuite.scala | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 8d4f72190fc4d..28b6aac861983 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -56,6 +56,20 @@ class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) } +class MyCheckpointRDD( + sc: SparkContext, + numPartitions: Int, + dependencies: List[Dependency[_]], + locations: Seq[Seq[String]] = Nil, + @(transient @param) tracker: MapOutputTrackerMaster = null, + indeterminate: Boolean = false) + extends MyRDD(sc, numPartitions, dependencies, locations, tracker, indeterminate) { + + // Allow doCheckpoint() on this RDD. + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + Iterator.empty +} + /** * An RDD for passing to DAGScheduler. These RDDs will use the dependencies and * preferredLocations (if any) that are passed to them. They are deliberately not executable @@ -2745,17 +2759,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("SPARK-23207: local checkpoint fail to rollback (checkpointed before)") { - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) shuffleMapRdd.localCheckpoint() - submit(shuffleMapRdd, Array(1, 1)) - completeNextResultStageWithSuccess(0, 0) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assertDataStructuresEmpty() + shuffleMapRdd.doCheckpoint() assertResultStageFailToRollback(shuffleMapRdd) } test("SPARK-23207: local checkpoint fail to rollback (checkpointing now)") { - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) shuffleMapRdd.localCheckpoint() assertResultStageFailToRollback(shuffleMapRdd) } @@ -2796,20 +2807,17 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi test("SPARK-23207: reliable checkpoint can avoid rollback (checkpointed before)") { sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) shuffleMapRdd.checkpoint() - submit(shuffleMapRdd, Array(1, 1)) - completeNextResultStageWithSuccess(0, 0) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assertDataStructuresEmpty() + shuffleMapRdd.doCheckpoint() assertResultStageNotRollbacked(shuffleMapRdd) } - test("SPARK-23207: reliable checkpoint can avoid rollback (checkpointing now)") { + test("SPARK-23207: reliable checkpoint fail to rollback (checkpointing now)") { sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) shuffleMapRdd.checkpoint() - assertResultStageNotRollbacked(shuffleMapRdd) + assertResultStageFailToRollback(shuffleMapRdd) } /** From a313403f9d382af8e2ff0314098b396920c5dbde Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 29 Aug 2018 01:06:56 +0800 Subject: [PATCH 21/26] update --- .../apache/spark/rdd/LocalCheckpointRDD.scala | 1 + .../main/scala/org/apache/spark/rdd/RDD.scala | 17 ++++++++--------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala index cd04737b816fa..6a19320dae398 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala @@ -33,6 +33,7 @@ import org.apache.spark.storage.RDDBlockId * @param sc the active SparkContext * @param rddId the ID of the checkpointed RDD * @param numPartitions the number of partitions in the checkpointed RDD + * @param deterministicLevel the DeterministicLevel of the local checkpointed RDD */ private[spark] class LocalCheckpointRDD[T: ClassTag]( sc: SparkContext, diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 73f15accd2b6f..52fd2cd9e1756 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1993,16 +1993,15 @@ object RDD { } /** - * The deterministic level of RDD's output (i.e. what `RDD#compute` returns), which indicates how - * the output will diff when Spark reruns the tasks for the RDD. There are 3 deterministic levels, - * ordered by the determinism from high to low: - * 1. DETERMINATE: The RDD output is always same (including order) when rerun. - * 2. UNORDERED: The RDD output is always the same data set but in potentially a different order - * when rerun. - * 3. INDETERMINATE. The RDD output can be different (not only order) when rerun. + * The deterministic level of RDD's output (i.e. what `RDD#compute` returns). This explains how + * the output will diff when Spark reruns the tasks for the RDD. There are 3 deterministic levels: + * 1. DETERMINATE: The RDD output is always the same data set in the same order after a rerun. + * 2. UNORDERED: The RDD output is always the same data set but the order can be different + * after a rerun. + * 3. INDETERMINATE. The RDD output can be different after a rerun. * - * Note that, the output of an RDD usually relies on parent RDDs. When a parent RDD's output is - * INDETERMINATE, it's very likely this RDD's output is also INDETERMINATE. + * Note that, the output of an RDD usually relies on the parent RDDs. When the parent RDD's output + * is INDETERMINATE, it's very likely the RDD's output is also INDETERMINATE. */ private[spark] object DeterministicLevel extends Enumeration { val DETERMINATE, UNORDERED, INDETERMINATE = Value From aa06d1e820090308597829eb4d9e1b63ac292ae0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Aug 2018 01:34:07 +0800 Subject: [PATCH 22/26] cleanup --- .../org/apache/spark/rdd/LocalCheckpointRDD.scala | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala index 6a19320dae398..503aa0dffc9f3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala @@ -33,17 +33,15 @@ import org.apache.spark.storage.RDDBlockId * @param sc the active SparkContext * @param rddId the ID of the checkpointed RDD * @param numPartitions the number of partitions in the checkpointed RDD - * @param deterministicLevel the DeterministicLevel of the local checkpointed RDD */ private[spark] class LocalCheckpointRDD[T: ClassTag]( sc: SparkContext, rddId: Int, - numPartitions: Int, - deterministicLevel: DeterministicLevel.Value) + numPartitions: Int) extends CheckpointRDD[T](sc) { def this(rdd: RDD[T]) { - this(rdd.context, rdd.id, rdd.partitions.length, rdd.outputDeterministicLevel) + this(rdd.context, rdd.id, rdd.partitions.length) } protected override def getPartitions: Array[Partition] = { @@ -66,9 +64,4 @@ private[spark] class LocalCheckpointRDD[T: ClassTag]( s"instead, which is slower than local checkpointing but more fault-tolerant.") } - // Local checkpoint is not reliable, we may still get output from original RDD, so the output - // deterministic level should also inherent from the original RDD. - override protected def getOutputDeterministicLevel = { - deterministicLevel - } } From 31a62bd1502af8eeb11e9489e778489fcd464df1 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 29 Aug 2018 01:41:35 +0800 Subject: [PATCH 23/26] update --- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 10 +++++----- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 12 ++++++------ .../sql/execution/exchange/ShuffleExchangeExec.scala | 6 +++--- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 64bd7199c2392..aa61997122cf4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -32,16 +32,16 @@ import org.apache.spark.{Partition, TaskContext} * doesn't modify the keys. * @param isFromBarrier Indicates whether this RDD is transformed from an RDDBarrier, a stage * containing at least one RDDBarrier shall be turned into a barrier stage. - * @param orderSensitiveFunc whether or not the function is order-sensitive. If it's order - * sensitive, it may return totally different result if the input order - * changed. Mostly stateful functions are order-sensitive. + * @param isOrderSensitive whether or not the function is order-sensitive. If it's order + * sensitive, it may return totally different result when the input order + * is changed. Mostly stateful functions are order-sensitive. */ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( var prev: RDD[T], f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false, isFromBarrier: Boolean = false, - orderSensitiveFunc: Boolean = false) + isOrderSensitive: Boolean = false) extends RDD[U](prev) { override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None @@ -60,7 +60,7 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( isFromBarrier || dependencies.exists(_.rdd.isBarrier()) override protected def getOutputDeterministicLevel = { - if (orderSensitiveFunc && prev.outputDeterministicLevel == DeterministicLevel.UNORDERED) { + if (isOrderSensitive && prev.outputDeterministicLevel == DeterministicLevel.UNORDERED) { DeterministicLevel.INDETERMINATE } else { super.getOutputDeterministicLevel diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 52fd2cd9e1756..fd5817ed464b5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -463,7 +463,7 @@ abstract class RDD[T: ClassTag]( // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( new ShuffledRDD[Int, T, T]( - mapPartitionsWithIndexInternal(distributePartition, orderSensitiveFunc = true), + mapPartitionsWithIndexInternal(distributePartition, isOrderSensitive = true), new HashPartitioner(numPartitions)), numPartitions, partitionCoalescer).values @@ -810,19 +810,19 @@ abstract class RDD[T: ClassTag]( * @param preservesPartitioning indicates whether the input function preserves the partitioner, * which should be `false` unless this is a pair RDD and the input * function doesn't modify the keys. - * @param orderSensitiveFunc whether or not the function is order-sensitive. If it's order - * sensitive, it may return totally different result if the input order - * changed. Mostly stateful functions are order-sensitive. + * @param isOrderSensitive whether or not the function is order-sensitive. If it's order + * sensitive, it may return totally different result when the input order + * is changed. Mostly stateful functions are order-sensitive. */ private[spark] def mapPartitionsWithIndexInternal[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false, - orderSensitiveFunc: Boolean = false): RDD[U] = withScope { + isOrderSensitive: Boolean = false): RDD[U] = withScope { new MapPartitionsRDD( this, (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), preservesPartitioning = preservesPartitioning, - orderSensitiveFunc = orderSensitiveFunc) + isOrderSensitive = isOrderSensitive) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 5fcbce98aaa34..9576605b1a214 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -307,18 +307,18 @@ object ShuffleExchangeExec { } // round-robin function is order sensitive if we don't sort the input. - val orderSensitiveFunc = isRoundRobin && !SQLConf.get.sortBeforeRepartition + val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition if (needToCopyObjectsBeforeShuffle(part)) { newRdd.mapPartitionsWithIndexInternal((_, iter) => { val getPartitionKey = getPartitionKeyExtractor() iter.map { row => (part.getPartition(getPartitionKey(row)), row.copy()) } - }, orderSensitiveFunc = orderSensitiveFunc) + }, isOrderSensitive = isOrderSensitive) } else { newRdd.mapPartitionsWithIndexInternal((_, iter) => { val getPartitionKey = getPartitionKeyExtractor() val mutablePair = new MutablePair[Int, InternalRow]() iter.map { row => mutablePair.update(part.getPartition(getPartitionKey(row)), row) } - }, orderSensitiveFunc = orderSensitiveFunc) + }, isOrderSensitive = isOrderSensitive) } } From ee52bfb7c6d2dc3e91191e9c2eb850fd98f05e84 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 1 Sep 2018 22:26:22 +0800 Subject: [PATCH 24/26] fix test --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 28b6aac861983..345cff58af2c0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2708,7 +2708,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId1 => stage }.head.findMissingPartitions() == Seq(0)) - // The second shuffle map stage is entirely rollbacked, because the root RDD is indeterminate. + // The second shuffle map stage is entirely rollbacked, because its succeeding map stage is + // indeterminate. assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0, 1)) @@ -2717,9 +2718,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // complete the first task of first shuffle map stage runEvent(makeCompletionEvent( - taskSets(0).tasks(0), Success, makeMapStatus("hostA", 2))) - // Complete the map stage. - completeShuffleMapStageSuccessfully(0, 1, numShufflePartitions = 2) + taskSets.last.tasks(0), Success, makeMapStatus("hostA", 2))) + // Complete the second map stage. completeShuffleMapStageSuccessfully(1, 2, numShufflePartitions = 2) // Complete the result stage. completeNextResultStageWithSuccess(2, 1) From 9a3b8f42c6f9f992fa870e0c7e35ef4be533b561 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 3 Sep 2018 10:59:35 +0800 Subject: [PATCH 25/26] always fail the job --- .../apache/spark/scheduler/DAGScheduler.scala | 35 +++++++++---------- .../spark/scheduler/DAGSchedulerSuite.scala | 28 ++------------- 2 files changed, 19 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4001bafd3d816..69c83ac6316f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1513,37 +1513,34 @@ private[spark] class DAGScheduler( } } + def generateErrorMessage(stage: Stage): String = { + "A shuffle map stage with indeterminate output was failed and retried. " + + s"However, Spark cannot rollback the $stage to re-process the input data, " + + "and has to fail this job. Please eliminate the determinism by checkpointing " + + "the RDD before repartition and try again." + } + activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) stagesToRollback.foreach { case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - val reason = "preceding shuffle map stage with indeterminate output gets " + - "retried." - try { - taskScheduler.killAllTaskAttempts( - mapStage.id, interruptThread = false, reason) - } catch { - case e: UnsupportedOperationException => - logWarning(s"Could not kill all tasks for $mapStage", e) - } - markStageAsFinished(mapStage, Some(reason), willRetry = true) - mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) - failedStages += mapStage + // TODO: support to rollback shuffle files. + // Currently the shuffle writing is "first write wins", so we can't re-run a + // shuffle map stage and overwrite existing shuffle files. We have to finish + // SPARK-8029 first. + abortStage(mapStage, generateErrorMessage(mapStage), None) } - case resultStage => + case resultStage: ResultStage if resultStage.activeJob.isDefined => val numMissingPartitions = resultStage.findMissingPartitions().length if (numMissingPartitions < resultStage.numTasks) { // TODO: support to rollback result tasks. - val errorMessage = "A shuffle map stage with indeterminate output was " + - s"failed and retried. However, Spark cannot rollback the $resultStage " + - "to re-process the input data, and has to fail this job. Please " + - "eliminate the determinism by checkpointing the RDD before " + - "repartition/zip and try again." - abortStage(failedStage, errorMessage, None) + abortStage(resultStage, generateErrorMessage(resultStage), None) } + + case _ => } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 345cff58af2c0..2dd888318c347 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2683,7 +2683,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), null)) - var failedStages = scheduler.failedStages.toSeq + val failedStages = scheduler.failedStages.toSeq assert(failedStages.length == 2) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. assert(failedStages.collect { @@ -2702,30 +2702,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), null)) - failedStages = scheduler.failedStages.toSeq - assert(failedStages.length == 2) - // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd1` needs to retry. - assert(failedStages.collect { - case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId1 => stage - }.head.findMissingPartitions() == Seq(0)) - // The second shuffle map stage is entirely rollbacked, because its succeeding map stage is - // indeterminate. - assert(failedStages.collect { - case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage - }.head.findMissingPartitions() == Seq(0, 1)) - - scheduler.resubmitFailedStages() - - // complete the first task of first shuffle map stage - runEvent(makeCompletionEvent( - taskSets.last.tasks(0), Success, makeMapStatus("hostA", 2))) - // Complete the second map stage. - completeShuffleMapStageSuccessfully(1, 2, numShufflePartitions = 2) - // Complete the result stage. - completeNextResultStageWithSuccess(2, 1) - - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assertDataStructuresEmpty() + // The job should fail because Spark can't rollback the shuffle map stage. + assert(failure != null && failure.getMessage.contains("Spark cannot rollback")) } private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = { From df162ad584e212291a3bfa73ff571a855914fad8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 5 Sep 2018 13:57:50 +0800 Subject: [PATCH 26/26] fix typo --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 69c83ac6316f6..50c91da8b13d1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1516,8 +1516,8 @@ private[spark] class DAGScheduler( def generateErrorMessage(stage: Stage): String = { "A shuffle map stage with indeterminate output was failed and retried. " + s"However, Spark cannot rollback the $stage to re-process the input data, " + - "and has to fail this job. Please eliminate the determinism by checkpointing " + - "the RDD before repartition and try again." + "and has to fail this job. Please eliminate the indeterminacy by " + + "checkpointing the RDD before repartition and try again." } activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil))