From 3445646cf0e24507c4c1c74cbca96db59c620008 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 20 Feb 2025 14:19:35 -0800 Subject: [PATCH 01/48] SPARK-51016. Fixing the code of isInDeterminate boolean for a Stage --- .../scala/org/apache/spark/Dependency.scala | 22 +++++-- .../main/scala/org/apache/spark/rdd/RDD.scala | 3 + .../spark/scheduler/ShuffleMapStage.scala | 2 + python/pyspark/pandas/internal.py | 6 +- .../sql/catalyst/expressions/Expression.scala | 3 + .../expressions/namedExpressions.scala | 16 ++++- .../expressions/ExpressionEvalHelper.scala | 2 +- .../expressions/NondeterministicSuite.scala | 34 +++++++++++ .../exchange/ShuffleExchangeExec.scala | 13 +++- .../spark/scheduler/ShuffleMapStageTest.scala | 61 +++++++++++++++++++ ...rojectedOrderingAndPartitioningSuite.scala | 40 +++++++++++- 11 files changed, 187 insertions(+), 15 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 573608c4327e..2751d7b3b2e0 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -79,13 +79,25 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( @transient private val _rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, - 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 shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor) + val serializer: Serializer, + val keyOrdering: Option[Ordering[K]], + val aggregator: Option[Aggregator[K, V, C]], + val mapSideCombine: Boolean, + val shuffleWriterProcessor: ShuffleWriteProcessor, + val isInDeterministic: Boolean) extends Dependency[Product2[K, V]] with Logging { + def this ( + rdd: RDD[_ <: Product2[K, V]], + partitioner: Partitioner, + serializer: Serializer = SparkEnv.get.serializer, + keyOrdering: Option[Ordering[K]] = None, + aggregator: Option[Aggregator[K, V, C]] = None, + mapSideCombine: Boolean = false, + shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor + ) = this(rdd, partitioner, serializer, keyOrdering, aggregator, mapSideCombine, + shuffleWriterProcessor, false) + if (mapSideCombine) { require(aggregator.isDefined, "Map-side combine without Aggregator specified!") } 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 80db818b77e4..fb8564827b02 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -2105,6 +2105,9 @@ abstract class RDD[T: ClassTag]( val deterministicLevelCandidates = dependencies.map { // The shuffle is not really happening, treat it like narrow dependency and assume the output // deterministic level of current RDD is same as parent. + case dep: ShuffleDependency[_, _, _] if dep.isInDeterministic => + DeterministicLevel.INDETERMINATE + case dep: ShuffleDependency[_, _, _] if dep.rdd.partitioner.exists(_ == dep.partitioner) => dep.rdd.outputDeterministicLevel diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index db09d19d0acf..38da37751206 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -94,4 +94,6 @@ private[spark] class ShuffleMapStage( .findMissingPartitions(shuffleDep.shuffleId) .getOrElse(0 until numPartitions) } + + override def isIndeterminate: Boolean = this.shuffleDep.isInDeterministic || super.isIndeterminate } diff --git a/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py index 3f6831b60067..bda05017e135 100644 --- a/python/pyspark/pandas/internal.py +++ b/python/pyspark/pandas/internal.py @@ -766,8 +766,9 @@ def __init__( for index_field, struct_field in zip(index_fields, struct_fields) ), (index_fields, struct_fields) else: + # TODO(SPARK-42965): For some reason, the metadata of StructField is different assert all( - index_field.struct_field == struct_field + _drop_metadata(index_field.struct_field) == _drop_metadata(struct_field) for index_field, struct_field in zip(index_fields, struct_fields) ), (index_fields, struct_fields) @@ -794,8 +795,9 @@ def __init__( for data_field, struct_field in zip(data_fields, struct_fields) ), (data_fields, struct_fields) else: + # TODO(SPARK-42965): For some reason, the metadata of StructField is different assert all( - data_field.struct_field == struct_field + _drop_metadata(data_field.struct_field) == _drop_metadata(struct_field) for data_field, struct_field in zip(data_fields, struct_fields) ), (data_fields, struct_fields) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 7d4f8c3b2564..2a6d928f5ab2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -115,6 +115,9 @@ abstract class Expression extends TreeNode[Expression] { */ lazy val deterministic: Boolean = children.forall(_.deterministic) + lazy val exprValHasIndeterministicCharacter: Boolean = !deterministic || + this.references.exists(_.exprValHasIndeterministicCharacter) + def nullable: Boolean /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 2af6a1ba84ec..421c2fd190a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -111,6 +111,10 @@ abstract class Attribute extends LeafExpression with NamedExpression { @transient override lazy val references: AttributeSet = AttributeSet(this) + override lazy val exprValHasIndeterministicCharacter: Boolean = + metadata.contains(Attribute.KEY_HAS_INDETERMINISTIC_COMPONENT) && + metadata.getBoolean(Attribute.KEY_HAS_INDETERMINISTIC_COMPONENT) + def withNullability(newNullability: Boolean): Attribute def withQualifier(newQualifier: Seq[String]): Attribute def withName(newName: String): Attribute @@ -123,6 +127,10 @@ abstract class Attribute extends LeafExpression with NamedExpression { } +object Attribute { + val KEY_HAS_INDETERMINISTIC_COMPONENT = "hasIndeterministicComponent" +} + /** * Used to assign a new name to a computation. * For example the SQL expression "1 + 1 AS a" could be represented as follows: @@ -194,7 +202,13 @@ case class Alias(child: Expression, name: String)( override def toAttribute: Attribute = { if (resolved) { - AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifier) + val mdForAttrib = if (this.exprValHasIndeterministicCharacter) { + new MetadataBuilder().withMetadata(metadata). + putBoolean(Attribute.KEY_HAS_INDETERMINISTIC_COMPONENT, true).build() + } else { + metadata + } + AttributeReference(name, child.dataType, child.nullable, mdForAttrib)(exprId, qualifier) } else { UnresolvedAttribute.quoted(name) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 184f5a2a9485..2347410d4537 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -76,7 +76,7 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB case _ => expr.mapChildren(replace) } - private def prepareEvaluation(expression: Expression): Expression = { + def prepareEvaluation(expression: Expression): Expression = { val serializer = new JavaSerializer(new SparkConf()).newInstance() val resolver = ResolveTimeZone val expr = replace(resolver.resolveTimeZones(expression)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala index bf1c930c0bd0..983810a5fdae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, KeyGroupedPartitioning, RangePartitioning} class NondeterministicSuite extends SparkFunSuite with ExpressionEvalHelper { test("MonotonicallyIncreasingID") { @@ -31,4 +32,37 @@ class NondeterministicSuite extends SparkFunSuite with ExpressionEvalHelper { test("InputFileName") { checkEvaluation(InputFileName(), "") } + + test("SPARK-51016: has Indeterministic Component") { + def assertIndeterminancyComponent(expression: Expression): Unit = + assert(prepareEvaluation(expression).exprValHasIndeterministicCharacter) + + assertIndeterminancyComponent(MonotonicallyIncreasingID()) + val alias = Alias(Multiply(MonotonicallyIncreasingID(), Literal(100L)), "al1")() + assertIndeterminancyComponent(alias) + assertIndeterminancyComponent(alias.toAttribute) + assertIndeterminancyComponent(Multiply(alias.toAttribute, Literal(1000L))) + assertIndeterminancyComponent( + HashPartitioning(Seq(Multiply(MonotonicallyIncreasingID(), Literal(100L))), 5)) + assertIndeterminancyComponent(HashPartitioning(Seq(alias.toAttribute), 5)) + assertIndeterminancyComponent( + RangePartitioning(Seq(SortOrder.apply(alias.toAttribute, Descending)), 5)) + assertIndeterminancyComponent(KeyGroupedPartitioning(Seq(alias.toAttribute), 5)) + } + + test("SPARK-51016: has Deterministic Component") { + def assertNoIndeterminancyComponent(expression: Expression): Unit = + assert(!prepareEvaluation(expression).exprValHasIndeterministicCharacter) + + assertNoIndeterminancyComponent(Literal(1000L)) + val alias = Alias(Multiply(Literal(10000L), Literal(100L)), "al1")() + assertNoIndeterminancyComponent(alias) + assertNoIndeterminancyComponent(alias.toAttribute) + assertNoIndeterminancyComponent( + HashPartitioning(Seq(Multiply(Literal(10L), Literal(100L))), 5)) + assertNoIndeterminancyComponent(HashPartitioning(Seq(alias.toAttribute), 5)) + assertNoIndeterminancyComponent( + RangePartitioning(Seq(SortOrder.apply(alias.toAttribute, Descending)), 5)) + assertNoIndeterminancyComponent(KeyGroupedPartitioning(Seq(alias.toAttribute), 5)) + } } 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 31a3f53eb719..2e6261de4040 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 @@ -30,7 +30,7 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProcessor} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Expression, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.logical.Statistics @@ -467,7 +467,10 @@ object ShuffleExchangeExec { }, isOrderSensitive = isOrderSensitive) } } - + val isIndeterministic = newPartitioning match { + case expr: Expression => expr.exprValHasIndeterministicCharacter + case _ => false + } // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds // are in the form of (partitionId, row) and every partitionId is in the expected range // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough. @@ -476,7 +479,11 @@ object ShuffleExchangeExec { rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), serializer, - shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics)) + None, + None, + false, + shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics), + isIndeterministic) dependency } diff --git a/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala b/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala new file mode 100644 index 000000000000..da3f619c4a2c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.sql.Encoders +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.LongType + +class ShuffleMapStageTest extends SharedSparkSession { + + test("SPARK-51016: ShuffleMapStage using indeterministic join keys should be INDETERMINATE") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val outerDf = spark.createDataset( + Seq((1L, "aa")))( + Encoders.tuple(Encoders.scalaLong, Encoders.STRING)).toDF("pkLeftt", "strleft") + + val innerDf = spark.createDataset( + Seq((1L, "11"), (2L, "22")))( + Encoders.tuple(Encoders.scalaLong, Encoders.STRING)).toDF("pkRight", "strright") + + val leftOuter = outerDf.select( + col("strleft"), when(isnull(col("pkLeftt")), floor(rand() * Literal(10000000L)). + cast(LongType)). + otherwise(col("pkLeftt")).as("pkLeft")) + + val outerjoin = leftOuter.hint("shuffle_hash"). + join(innerDf, col("pkLeft") === col("pkRight"), "left_outer") + val shuffleStages: Array[ShuffleMapStage] = Array.ofDim(2) + spark.sparkContext.addSparkListener(new SparkListener() { + var i = 0 + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + if (stageSubmitted.stageInfo.shuffleDepId.isDefined) { + shuffleStages(i) = + spark.sparkContext.dagScheduler.shuffleIdToMapStage(stageSubmitted.stageInfo.stageId) + i +=1 + } + } + }); + outerjoin.collect() + assert(shuffleStages.filter(_.isIndeterminate).size == 1) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala index ec13d48d45f8..395c03b81055 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala @@ -17,14 +17,17 @@ package org.apache.spark.sql.execution -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{DeterministicLevel, RDD} +import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec +import org.apache.spark.sql.functions.{col, floor, isnull, rand, when} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{LongType, StringType} class ProjectedOrderingAndPartitioningSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { @@ -210,6 +213,37 @@ class ProjectedOrderingAndPartitioningSuite assert(outputOrdering.head.child.asInstanceOf[Attribute].name == "a") assert(outputOrdering.head.sameOrderExpressions.size == 0) } + + test("SPARK-51016: ShuffleRDD using indeterministic join keys should be INDETERMINATE") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val outerDf = spark.createDataset( + Seq((1L, "aa"), (null, "aa"), (2L, "bb"), (null, "bb"), (3L, "cc"), (null, "cc")))( + Encoders.tupleEncoder(Encoders.LONG, Encoders.STRING)).toDF("pkLeftt", "strleft") + + val innerDf = spark.createDataset( + Seq((1L, "11"), (2L, "22"), (3L, "33")))( + Encoders.tupleEncoder(Encoders.LONG, Encoders.STRING)).toDF("pkRight", "strright") + + val leftOuter = outerDf.select( + col("strleft"), when(isnull(col("pkLeftt")), floor(rand() * Literal(10000000L)). + cast(LongType)). + otherwise(col("pkLeftt")).as("pkLeft")) + + val outerjoin = leftOuter.hint("shuffle_hash"). + join(innerDf, col("pkLeft") === col("pkRight"), "left_outer") + + outerjoin.collect() + val finalPlan = outerjoin.queryExecution.executedPlan + val shuffleHJExec = finalPlan.children(0).asInstanceOf[ShuffledHashJoinExec] + assert(shuffleHJExec.left.asInstanceOf[InputAdapter].execute().outputDeterministicLevel == + DeterministicLevel.INDETERMINATE) + + assert(shuffleHJExec.right.asInstanceOf[InputAdapter].execute().outputDeterministicLevel == + DeterministicLevel.UNORDERED) + + assert(shuffleHJExec.execute().outputDeterministicLevel == DeterministicLevel.INDETERMINATE) + } + } } private case class DummyLeafPlanExec(output: Seq[Attribute]) extends LeafExecNode { From 3ce5ad3168a1e55d254ded8392be1ce7ced45b84 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 20 Feb 2025 15:18:32 -0800 Subject: [PATCH 02/48] SPARK-51016. updating comment --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 fb8564827b02..2ccc0f6476af 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -2103,11 +2103,11 @@ abstract class RDD[T: ClassTag]( @DeveloperApi protected def getOutputDeterministicLevel: DeterministicLevel.Value = { val deterministicLevelCandidates = dependencies.map { - // The shuffle is not really happening, treat it like narrow dependency and assume the output - // deterministic level of current RDD is same as parent. case dep: ShuffleDependency[_, _, _] if dep.isInDeterministic => DeterministicLevel.INDETERMINATE + // The shuffle is not really happening, treat it like narrow dependency and assume the output + // deterministic level of current RDD is same as parent. case dep: ShuffleDependency[_, _, _] if dep.rdd.partitioner.exists(_ == dep.partitioner) => dep.rdd.outputDeterministicLevel From c537b3abd141e86b5af5d0eb31ee005bb50b08c8 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 20 Feb 2025 16:17:37 -0800 Subject: [PATCH 03/48] SPARK-51272. Fix for the race condition in Scheduler causing failure in retrying all partitions in case of indeterministic shuffle keys --- .../apache/spark/scheduler/DAGScheduler.scala | 239 +++++++++++------- .../apache/spark/scheduler/ResultStage.scala | 7 +- .../spark/scheduler/ShuffleMapStage.scala | 9 +- .../org/apache/spark/scheduler/Stage.scala | 68 ++++- .../spark/scheduler/DAGSchedulerSuite.scala | 14 +- 5 files changed, 230 insertions(+), 107 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 aee92ba928b4..a9efdd7ca1d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1898,25 +1898,51 @@ private[spark] class DAGScheduler( // Make sure the task's accumulators are updated before any other processing happens, so that // we can post a task end event before any jobs or stages are updated. The accumulators are // only updated in certain cases. - event.reason match { + val (readLockTaken, isIndeterministicZombie) = event.reason match { case Success => - task match { - case rt: ResultTask[_, _] => - val resultStage = stage.asInstanceOf[ResultStage] - resultStage.activeJob match { - case Some(job) => - // Only update the accumulator once for each result task. - if (!job.finished(rt.outputId)) { - updateAccumulators(event) - } - case None => // Ignore update if task's job has finished. - } - case _ => - updateAccumulators(event) + stage.acquireStageReadLock() + val isZombieIndeterminate = + (task.stageAttemptId < stage.latestInfo.attemptNumber() + && stage.isIndeterminate) || + stage.treatAllPartitionsMissing(task.stageAttemptId) + if (!isZombieIndeterminate) { + task match { + case rt: ResultTask[_, _] => + val resultStage = stage.asInstanceOf[ResultStage] + resultStage.activeJob match { + case Some(job) => + // Only update the accumulator once for each result task. + if (!job.finished(rt.outputId)) { + updateAccumulators(event) + } + case _ => // Ignore update if task's job has finished. + } + case _ => updateAccumulators(event) + } } - case _: ExceptionFailure | _: TaskKilled => updateAccumulators(event) - case _ => + (true, isZombieIndeterminate) + + case _: ExceptionFailure | _: TaskKilled => + updateAccumulators(event) + (false, false) + + case _ => (false, false) + } + + try { + handleTaskCompletionInOptionalReadLock(event, task, stageId, stage, isIndeterministicZombie) + } finally { + if (readLockTaken) { + stage.releaseStageReadLock() + } } + } + + private def handleTaskCompletionInOptionalReadLock( + event: CompletionEvent, + task: Task[_], stageId: Int, + stage: Stage, + isIndeterministicZombie: Boolean): Unit = { if (trackingCacheVisibility) { // Update rdd blocks' visibility status. blockManagerMaster.updateRDDBlockVisibility( @@ -1936,7 +1962,7 @@ private[spark] class DAGScheduler( } task match { - case rt: ResultTask[_, _] => + case rt: ResultTask[_, _] if !isIndeterministicZombie => // Cast to ResultStage here because it's part of the ResultTask // TODO Refactor this out to a function that accepts a ResultStage val resultStage = stage.asInstanceOf[ResultStage] @@ -1984,7 +2010,7 @@ private[spark] class DAGScheduler( logInfo(log"Ignoring result from ${MDC(RESULT, rt)} because its job has finished") } - case smt: ShuffleMapTask => + case smt: ShuffleMapTask if !isIndeterministicZombie => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] // Ignore task completion for old attempt of indeterminate stage val ignoreIndeterminate = stage.isIndeterminate && @@ -2017,6 +2043,8 @@ private[spark] class DAGScheduler( processShuffleMapStageCompletion(shuffleStage) } } + + case _ => // ignore } case FetchFailed(bmAddress, shuffleId, _, mapIndex, reduceId, failureMessage) => @@ -2121,92 +2149,111 @@ private[spark] class DAGScheduler( failedStages += failedStage failedStages += mapStage if (noResubmitEnqueued) { - // 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 UNORDERED, we are fine. The shuffle partitioner is - // guaranteed to be determinate, so the input data of the reducers will not change - // even if the map tasks are re-tried. - if (mapStage.isIndeterminate) { - // It's a little tricky to find all the succeeding stages of `mapStage`, 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 `mapStage`. 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 = HashSet[Stage](mapStage) - - 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) + val writeLockedStages = mutable.Buffer.empty[Stage] + try { + // 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 UNORDERED, we are fine. The shuffle partitioner is + // guaranteed to be determinate, so the input data of the reducers will not change + // even if the map tasks are re-tried. + if (mapStage.isIndeterminate) { + // It's a little tricky to find all the succeeding stages of `mapStage`, 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 `mapStage`. 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 = HashSet[Stage](mapStage) + + 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) + } } } - } - 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 indeterminacy by " + - "checkpointing the RDD before repartition and try again." - } + 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 indeterminacy by " + + "checkpointing the RDD before repartition and try again." + } + + activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) - activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) - - // The stages will be rolled back after checking - val rollingBackStages = HashSet[Stage](mapStage) - stagesToRollback.foreach { - case mapStage: ShuffleMapStage => - val numMissingPartitions = mapStage.findMissingPartitions().length - if (numMissingPartitions < mapStage.numTasks) { - if (sc.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { - val reason = "A shuffle map stage with indeterminate output was failed " + - "and retried. However, Spark can only do this while using the new " + - "shuffle block fetching protocol. Please check the config " + - "'spark.shuffle.useOldFetchProtocol', see more detail in " + - "SPARK-27665 and SPARK-25341." - abortStage(mapStage, reason, None) + // The stages will be rolled back after checking + val rollingBackStages = HashSet[Stage](mapStage) + stagesToRollback.foreach { + case mapStage: ShuffleMapStage => + if (mapStage.acquireStageWriteLock()) { + writeLockedStages += mapStage + } + val numMissingPartitions = mapStage.findMissingPartitions().length + if (numMissingPartitions < mapStage.numTasks) { + if (sc.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + val reason = "A shuffle map stage with indeterminate output was failed " + + "and retried. However, Spark can only do this while using the new " + + "shuffle block fetching protocol. Please check the config " + + "'spark.shuffle.useOldFetchProtocol', see more detail in " + + "SPARK-27665 and SPARK-25341." + abortStage(mapStage, reason, None) + } else { + rollingBackStages += mapStage + mapStage.markAttemptIdForAllPartitionsMissing( + mapStage.latestInfo.attemptNumber()) + } } else { - rollingBackStages += mapStage + mapStage.markAttemptIdForAllPartitionsMissing( + mapStage.latestInfo.attemptNumber()) } - } - case resultStage: ResultStage if resultStage.activeJob.isDefined => - val numMissingPartitions = resultStage.findMissingPartitions().length - if (numMissingPartitions < resultStage.numTasks) { - // TODO: support to rollback result tasks. - abortStage(resultStage, generateErrorMessage(resultStage), None) - } + case resultStage: ResultStage if resultStage.activeJob.isDefined => + if (resultStage.acquireStageWriteLock()) { + writeLockedStages += resultStage + } + val numMissingPartitions = resultStage.findMissingPartitions().length + if (numMissingPartitions < resultStage.numTasks) { + // TODO: support to rollback result tasks. + abortStage(resultStage, generateErrorMessage(resultStage), None) + } else { + resultStage.markAttemptIdForAllPartitionsMissing( + resultStage.latestInfo.attemptNumber()) + } - case _ => + case _ => + } + logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + + log"we will roll back and rerun below stages which include itself and all its " + + log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") } - logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + - log"we will roll back and rerun below stages which include itself and all its " + - log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") - } - // 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 - // messages by checking whether a resubmit is already in the event queue for the - // failed stage. If there is already a resubmit enqueued for a different failed - // stage, that event would also be sufficient to handle the current failed stage, but - // producing a resubmit for each failed stage makes debugging and logging a little - // simpler while not producing an overwhelming number of scheduler events. - logInfo( - log"Resubmitting ${MDC(STAGE, mapStage)} " + - log"(${MDC(STAGE_NAME, mapStage.name)}) and ${MDC(FAILED_STAGE, failedStage)} " + - log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) due to fetch failure") - messageScheduler.schedule( - new Runnable { - override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) - }, - DAGScheduler.RESUBMIT_TIMEOUT, - TimeUnit.MILLISECONDS - ) + // 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 + // messages by checking whether a resubmit is already in the event queue for the + // failed stage. If there is already a resubmit enqueued for a different failed + // stage, that event would also be sufficient to handle the current failed stage, but + // producing a resubmit for each failed stage makes debugging and logging a little + // simpler while not producing an overwhelming number of scheduler events. + logInfo( + log"Resubmitting ${MDC(STAGE, mapStage)} " + + log"(${MDC(STAGE_NAME, mapStage.name)}) and ${MDC(FAILED_STAGE, failedStage)} " + + log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) due to fetch failure") + messageScheduler.schedule( + new Runnable { + override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) + }, + DAGScheduler.RESUBMIT_TIMEOUT, + TimeUnit.MILLISECONDS + ) + } finally { + writeLockedStages.foreach(_.releaseStageWriteLock()) + } } } @@ -2263,8 +2310,8 @@ private[spark] class DAGScheduler( log"and there is a more recent attempt for that stage (attempt " + log"${MDC(NUM_ATTEMPT, failedStage.latestInfo.attemptNumber())}) running") } else { - logInfo(log"Marking ${MDC(STAGE_ID, failedStage.id)} (${MDC(STAGE_NAME, failedStage.name)}) " + - log"as failed due to a barrier task failed.") + logInfo(log"Marking ${MDC(STAGE_ID, failedStage.id)} (${MDC(STAGE_NAME, failedStage.name)}) " + + log"as failed due to a barrier task failed.") val message = s"Stage failed because barrier task $task finished unsuccessfully.\n" + failure.toErrorString try { 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 7fdc3186e86b..97130ddd7455 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -61,7 +61,12 @@ private[spark] class ResultStage( */ override def findMissingPartitions(): Seq[Int] = { val job = activeJob.get - (0 until job.numPartitions).filter(id => !job.finished(id)) + val allPartitions = (0 until job.numPartitions) + if (this.treatAllPartitionsMissing(this.latestInfo.attemptNumber())) { + allPartitions + } else { + allPartitions.filter(id => !job.finished(id)) + } } override def toString: String = "ResultStage " + id diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index db09d19d0acf..57d53b89d397 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -90,8 +90,11 @@ private[spark] class ShuffleMapStage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ override def findMissingPartitions(): Seq[Int] = { - mapOutputTrackerMaster - .findMissingPartitions(shuffleDep.shuffleId) - .getOrElse(0 until numPartitions) + if (this.treatAllPartitionsMissing(this.latestInfo.attemptNumber())) { + 0 until numPartitions + } else { + mapOutputTrackerMaster + .findMissingPartitions(shuffleDep.shuffleId).getOrElse(0 until numPartitions) + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index f35beafd8748..146b34ed8285 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.locks.ReentrantReadWriteLock + import scala.collection.mutable.HashSet import org.apache.spark.executor.TaskMetrics @@ -63,6 +65,12 @@ private[scheduler] abstract class Stage( val resourceProfileId: Int) extends Logging { + @volatile + private var attemptIdAllPartitionsMissing: Int = -1 + + private val stageReattemptLock = new ReentrantReadWriteLock() + private val stageReadLock = stageReattemptLock.readLock() + private val stageWriteLock = stageReattemptLock.writeLock() val numPartitions = rdd.partitions.length /** Set of jobs that this stage belongs to. */ @@ -100,12 +108,21 @@ private[scheduler] abstract class Stage( def makeNewStageAttempt( numPartitionsToCompute: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = { - val metrics = new TaskMetrics - metrics.register(rdd.sparkContext) - _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, - resourceProfileId = resourceProfileId) - nextAttemptId += 1 + val writeLockTaken = this.acquireStageWriteLock() + try { + val metrics = new TaskMetrics + metrics.register(rdd.sparkContext) + _latestInfo = StageInfo.fromStage( + this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, + resourceProfileId = resourceProfileId) + nextAttemptId += 1 + // clear the entry in the allPartitionsAsMissing set + attemptIdAllPartitionsMissing = -1 + } finally { + if (writeLockTaken) { + this.releaseStageWriteLock() + } + } } /** Forward the nextAttemptId if skipped and get visited for the first time. */ @@ -131,4 +148,43 @@ private[scheduler] abstract class Stage( def isIndeterminate: Boolean = { rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE } + + def treatAllPartitionsMissing(attemptId: Int): Boolean = + this.attemptIdAllPartitionsMissing == attemptId + + def markAttemptIdForAllPartitionsMissing(attemptId: Int): Unit = + this.attemptIdAllPartitionsMissing = attemptId + + def acquireStageReadLock(): Unit = { + this.stageReadLock.lockInterruptibly() + val prevSet = Stage.threadHoldingReadLock.get() + Stage.threadHoldingReadLock.set(prevSet + this.id) + } + + def releaseStageReadLock(): Unit = { + val prevSet = Stage.threadHoldingReadLock.get() + Stage.threadHoldingReadLock.set(prevSet - this.id) + this.stageReadLock.unlock() + } + + def acquireStageWriteLock(): Boolean = { + if (Stage.threadHoldingReadLock.get().contains(this.id)) { + false + } else { + stageWriteLock.lockInterruptibly() + true + } + } + + def releaseStageWriteLock(): Unit = { + stageWriteLock.unlock() + } +} + +object Stage { + private val threadHoldingReadLock = new ThreadLocal[Set[Int]] { + override protected def initialValue(): Set[Int] = { + Set.empty[Int] + } + } } 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 3e507df706ba..111bb25ce62e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3192,9 +3192,14 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val failedStages = scheduler.failedStages.toSeq assert(failedStages.map(_.id) == Seq(1, 2)) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. + // TODO: Asif THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all + // the partitions need to be retried + /* assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage + }.head.findMissingPartitions() == Seq(0)) */ assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage - }.head.findMissingPartitions() == Seq(0)) + }.head.findMissingPartitions() == Seq(0, 1)) // The result stage is still waiting for its 2 tasks to complete assert(failedStages.collect { case stage: ResultStage => stage @@ -4163,9 +4168,16 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val failedStages = scheduler.failedStages.toSeq assert(failedStages.map(_.id) == Seq(1, 2)) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. + // TODO: Asif THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all + // the partitions need to be retried + /* assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0)) + */ + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage + }.head.findMissingPartitions() == Seq(0, 1)) // The result stage is still waiting for its 2 tasks to complete assert(failedStages.collect { case stage: ResultStage => stage From 765ae5559b98dbb15b849f515a4e45627c6fd0cc Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 21 Feb 2025 16:16:54 -0800 Subject: [PATCH 04/48] SPARK-51272. added a reliable bug test demonstrating the race condition which will fail if all partitions are not retried for indeterministic shuffle stage --- .../spark/scheduler/DAGSchedulerSuite.scala | 118 +++++++++++++++++- 1 file changed, 116 insertions(+), 2 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 111bb25ce62e..8ffa9243a4aa 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -81,6 +81,15 @@ class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) } } + def postInCurrentThread(event: DAGSchedulerEvent): Unit = { + try { + // Forward event to `onReceive` directly to avoid processing event asynchronously. + onReceive(event) + } catch { + case NonFatal(e) => onError(e) + } + } + override def onError(e: Throwable): Unit = { logError("Error in DAGSchedulerEventLoop: ", e) dagScheduler.stop() @@ -175,6 +184,11 @@ class DummyScheduledFuture( class DAGSchedulerSuiteDummyException extends Exception +trait DagSchedulerInterceptor { + def interceptHandleTaskCompletion(event: CompletionEvent): Unit = {} + def interceptResubmitFailedStages(): Unit = {} +} + class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with TimeLimits { import DAGSchedulerSuite._ @@ -300,6 +314,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } var sparkListener: EventInfoRecordingListener = null + var dagSchedulerInterceptor: DagSchedulerInterceptor = null var blockManagerMaster: BlockManagerMaster = null var mapOutputTracker: MapOutputTrackerMaster = null @@ -367,7 +382,8 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti env: SparkEnv, clock: Clock = new SystemClock(), shuffleMergeFinalize: Boolean = true, - shuffleMergeRegister: Boolean = true + shuffleMergeRegister: Boolean = true, + dagSchedulerInterceptorOpt: Option[DagSchedulerInterceptor] = None ) extends DAGScheduler( sc, taskScheduler, listenerBus, mapOutputTracker, blockManagerMaster, env, clock) { /** @@ -399,12 +415,18 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } override private[scheduler] def handleTaskCompletion(event: CompletionEvent): Unit = { + dagSchedulerInterceptorOpt.foreach(_.interceptHandleTaskCompletion(event)) super.handleTaskCompletion(event) runningTaskInfos.get(event.task.stageId).foreach{ partitions => partitions -= event.task.partitionId if (partitions.isEmpty) runningTaskInfos.remove(event.task.stageId) } } + + override private[scheduler] def resubmitFailedStages(): Unit = { + dagSchedulerInterceptorOpt.foreach(_.interceptResubmitFailedStages()) + super.resubmitFailedStages() + } } override def beforeEach(): Unit = { @@ -442,7 +464,9 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti sc.listenerBus, mapOutputTracker, blockManagerMaster, - sc.env)) + sc.env, + dagSchedulerInterceptorOpt = Option(dagSchedulerInterceptor) + )) dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler) } @@ -453,6 +477,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti dagEventProcessLoopTester.stop() mapOutputTracker.stop() broadcastManager.stop() + this.dagSchedulerInterceptor = null } finally { super.afterEach() } @@ -481,6 +506,13 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti dagEventProcessLoopTester.post(event) } + private def runEventInCurrentThread(event: DAGSchedulerEvent): Unit = { + // Ensure the initialization of various components + sc + dagEventProcessLoopTester.asInstanceOf[DAGSchedulerEventProcessLoopTester]. + postInCurrentThread(event) + } + /** * When we submit dummy Jobs, this is the compute function we supply. Except in a local test * below, we do not expect this function to ever be executed; instead, we will return results @@ -3185,6 +3217,88 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti "Spark can only do this while using the new shuffle block fetching protocol")) } + test("SPARK-51272: retry all the succeeding stages when the map stage is indeterminate with" + + " concurrent tasks completion") { + if (scheduler != null) { + this.afterEach() + } + + val monitor = new Object() + val resubmitFailedStageReached = Array.fill[Boolean](1)(false) + this.dagSchedulerInterceptor = new DagSchedulerInterceptor { + override def interceptHandleTaskCompletion(event: CompletionEvent): Unit = { + event.reason match { + case Success if event.task.isInstanceOf[ResultTask[_, _]] => + assert(resubmitFailedStageReached(0)) + monitor.synchronized { + monitor.notify() + } + + case _ => + } + } + + override def interceptResubmitFailedStages(): Unit = { + monitor.synchronized { + resubmitFailedStageReached(0) = true + monitor.notify() + monitor.wait() + } + } + } + + this.beforeEach() + + val numPartitions = 2 + val (shuffleId1, shuffleId2) = constructTwoIndeterminateStage() + completeShuffleMapStageSuccessfully(shuffleId2, 0, numPartitions) + val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] + val activeJob = resultStage.activeJob + assert(activeJob.isDefined) + // The result stage is still waiting for its 2 tasks to complete + assert(resultStage.findMissingPartitions() == Seq.tabulate(numPartitions)(i => i)) + new Thread(() => { + runEventInCurrentThread( + makeCompletionEvent( + taskSets(2).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), + null)) + }).start() + + monitor.synchronized { + if (!resubmitFailedStageReached(0)) { + monitor.wait() + } + } + assert(resubmitFailedStageReached(0)) + new Thread(() => { + runEventInCurrentThread(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) + }).start() + + val shuffleStage1 = this.scheduler.shuffleIdToMapStage(shuffleId1) + val shuffleStage2 = this.scheduler.shuffleIdToMapStage(shuffleId2) + var keepGoing = true + while (keepGoing) { + Thread.sleep(500) + keepGoing = shuffleStage1.latestInfo.attemptNumber() != 1 + } + completeShuffleMapStageSuccessfully(0, 1, numPartitions) + keepGoing = true + while (keepGoing) { + Thread.sleep(500) + keepGoing = shuffleStage2.latestInfo.attemptNumber() != 1 + } + + completeShuffleMapStageSuccessfully(1, 1, numPartitions) + keepGoing = true + while (keepGoing) { + Thread.sleep(500) + keepGoing = resultStage.latestInfo.attemptNumber() != 1 + } + + assert(resultStage.latestInfo.numTasks == 2) + } + test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { val (shuffleId1, shuffleId2) = constructIndeterminateStageFetchFailed() From e9a4659ec4a9b7fb79cc3a49cb70df9c9f1233be Mon Sep 17 00:00:00 2001 From: ashahid Date: Sun, 23 Feb 2025 11:15:30 -0800 Subject: [PATCH 05/48] SPARK-51272. fixed comment --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 8ffa9243a4aa..88175706bef8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3306,7 +3306,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val failedStages = scheduler.failedStages.toSeq assert(failedStages.map(_.id) == Seq(1, 2)) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. - // TODO: Asif THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all + // TODO: THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all // the partitions need to be retried /* assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage @@ -4282,7 +4282,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val failedStages = scheduler.failedStages.toSeq assert(failedStages.map(_.id) == Seq(1, 2)) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. - // TODO: Asif THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all + // TODO: THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all // the partitions need to be retried /* assert(failedStages.collect { From 038e14daf0b0a1007468712797a8cfffee7261ab Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 28 Feb 2025 13:42:00 -0800 Subject: [PATCH 06/48] SPARK-51016. Refactored the code so as to pass the boolean hasInDeterminism via curried constructor of AttributeReference, as per feedback. Renamed the boolean --- .../sql/catalyst/expressions/Expression.scala | 7 ++-- .../expressions/ProjectionOverSchema.scala | 2 +- .../expressions/namedExpressions.scala | 34 +++++++------------ .../expressions/NondeterministicSuite.scala | 4 +-- .../spark/sql/catalyst/plans/PlanTest.scala | 3 +- .../exchange/ShuffleExchangeExec.scala | 2 +- 6 files changed, 24 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 2a6d928f5ab2..4922713de408 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -115,8 +115,11 @@ abstract class Expression extends TreeNode[Expression] { */ lazy val deterministic: Boolean = children.forall(_.deterministic) - lazy val exprValHasIndeterministicCharacter: Boolean = !deterministic || - this.references.exists(_.exprValHasIndeterministicCharacter) + def hasIndeterminism: Boolean = _hasIndeterminism + + @transient + private lazy val _hasIndeterminism: Boolean = !deterministic || + this.references.exists(_.hasIndeterminism) def nullable: Boolean diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala index bb67c173b946..ea092093d095 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala @@ -38,7 +38,7 @@ case class ProjectionOverSchema(schema: StructType, output: AttributeSet) { private def getProjection(expr: Expression): Option[Expression] = expr match { case a: AttributeReference if fieldNames.contains(a.name) && output.contains(a) => - Some(a.copy(dataType = schema(a.name).dataType)(a.exprId, a.qualifier)) + Some(a.copy(dataType = schema(a.name).dataType)(a.exprId, a.qualifier, a.hasIndeterminism)) case GetArrayItem(child, arrayItemOrdinal, failOnError) => getProjection(child).map { projection => GetArrayItem(projection, arrayItemOrdinal, failOnError) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 421c2fd190a4..9c365d7565a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -111,9 +111,7 @@ abstract class Attribute extends LeafExpression with NamedExpression { @transient override lazy val references: AttributeSet = AttributeSet(this) - override lazy val exprValHasIndeterministicCharacter: Boolean = - metadata.contains(Attribute.KEY_HAS_INDETERMINISTIC_COMPONENT) && - metadata.getBoolean(Attribute.KEY_HAS_INDETERMINISTIC_COMPONENT) + override def hasIndeterminism: Boolean = false def withNullability(newNullability: Boolean): Attribute def withQualifier(newQualifier: Seq[String]): Attribute @@ -127,10 +125,6 @@ abstract class Attribute extends LeafExpression with NamedExpression { } -object Attribute { - val KEY_HAS_INDETERMINISTIC_COMPONENT = "hasIndeterministicComponent" -} - /** * Used to assign a new name to a computation. * For example the SQL expression "1 + 1 AS a" could be represented as follows: @@ -202,13 +196,8 @@ case class Alias(child: Expression, name: String)( override def toAttribute: Attribute = { if (resolved) { - val mdForAttrib = if (this.exprValHasIndeterministicCharacter) { - new MetadataBuilder().withMetadata(metadata). - putBoolean(Attribute.KEY_HAS_INDETERMINISTIC_COMPONENT, true).build() - } else { - metadata - } - AttributeReference(name, child.dataType, child.nullable, mdForAttrib)(exprId, qualifier) + AttributeReference(name, child.dataType, child.nullable)(exprId, qualifier, + this.hasIndeterminism) } else { UnresolvedAttribute.quoted(name) } @@ -288,7 +277,8 @@ case class AttributeReference( nullable: Boolean = true, override val metadata: Metadata = Metadata.empty)( val exprId: ExprId = NamedExpression.newExprId, - val qualifier: Seq[String] = Seq.empty[String]) + val qualifier: Seq[String] = Seq.empty[String], + override val hasIndeterminism: Boolean = false) extends Attribute with Unevaluable { override lazy val treePatternBits: BitSet = AttributeReferenceTreeBits.bits @@ -326,7 +316,8 @@ case class AttributeReference( } override def newInstance(): AttributeReference = - AttributeReference(name, dataType, nullable, metadata)(qualifier = qualifier) + AttributeReference(name, dataType, nullable, metadata)(qualifier = qualifier, + hasIndeterminism = hasIndeterminism) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -335,7 +326,8 @@ case class AttributeReference( if (nullable == newNullability) { this } else { - AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier) + AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier, + hasIndeterminism) } } @@ -343,7 +335,7 @@ case class AttributeReference( if (name == newName) { this } else { - AttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier) + AttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier, hasIndeterminism) } } @@ -354,7 +346,7 @@ case class AttributeReference( if (newQualifier == qualifier) { this } else { - AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier) + AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier, hasIndeterminism) } } @@ -362,12 +354,12 @@ case class AttributeReference( if (exprId == newExprId) { this } else { - AttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier) + AttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier, hasIndeterminism) } } override def withMetadata(newMetadata: Metadata): AttributeReference = { - AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier) + AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier, hasIndeterminism) } override def withDataType(newType: DataType): AttributeReference = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala index 983810a5fdae..2f0aa5fa6875 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala @@ -35,7 +35,7 @@ class NondeterministicSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-51016: has Indeterministic Component") { def assertIndeterminancyComponent(expression: Expression): Unit = - assert(prepareEvaluation(expression).exprValHasIndeterministicCharacter) + assert(prepareEvaluation(expression).hasIndeterminism) assertIndeterminancyComponent(MonotonicallyIncreasingID()) val alias = Alias(Multiply(MonotonicallyIncreasingID(), Literal(100L)), "al1")() @@ -52,7 +52,7 @@ class NondeterministicSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-51016: has Deterministic Component") { def assertNoIndeterminancyComponent(expression: Expression): Unit = - assert(!prepareEvaluation(expression).exprValHasIndeterministicCharacter) + assert(!prepareEvaluation(expression).hasIndeterminism) assertNoIndeterminancyComponent(Literal(1000L)) val alias = Alias(Multiply(Literal(10000L), Literal(100L)), "al1")() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index f06e6ed137cc..062a27075185 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -69,7 +69,8 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => protected def rewriteNameFromAttrNullability(plan: LogicalPlan): LogicalPlan = { plan.transformAllExpressions { case a @ AttributeReference(name, _, false, _) => - a.copy(name = s"*$name")(exprId = a.exprId, qualifier = a.qualifier) + a.copy(name = s"*$name")(exprId = a.exprId, qualifier = a.qualifier, + hasIndeterminism = a.hasIndeterminism) } } 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 2e6261de4040..bf63d2772c87 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 @@ -468,7 +468,7 @@ object ShuffleExchangeExec { } } val isIndeterministic = newPartitioning match { - case expr: Expression => expr.exprValHasIndeterministicCharacter + case expr: Expression => expr.hasIndeterminism case _ => false } // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds From 7f2d2bccd44836cee207de3e2cc4f304a0f1b804 Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 28 Feb 2025 13:48:57 -0800 Subject: [PATCH 07/48] SPARK-51016. Refactored the code so as to pass the boolean hasInDeterminism via curried constructor of AttributeReference, as per feedback. Renamed the boolean --- python/pyspark/pandas/internal.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py index bda05017e135..3f6831b60067 100644 --- a/python/pyspark/pandas/internal.py +++ b/python/pyspark/pandas/internal.py @@ -766,9 +766,8 @@ def __init__( for index_field, struct_field in zip(index_fields, struct_fields) ), (index_fields, struct_fields) else: - # TODO(SPARK-42965): For some reason, the metadata of StructField is different assert all( - _drop_metadata(index_field.struct_field) == _drop_metadata(struct_field) + index_field.struct_field == struct_field for index_field, struct_field in zip(index_fields, struct_fields) ), (index_fields, struct_fields) @@ -795,9 +794,8 @@ def __init__( for data_field, struct_field in zip(data_fields, struct_fields) ), (data_fields, struct_fields) else: - # TODO(SPARK-42965): For some reason, the metadata of StructField is different assert all( - _drop_metadata(data_field.struct_field) == _drop_metadata(struct_field) + data_field.struct_field == struct_field for data_field, struct_field in zip(data_fields, struct_fields) ), (data_fields, struct_fields) From 9cec75e1012c7d7bd9751b7c0982ea7046363727 Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 28 Feb 2025 14:33:45 -0800 Subject: [PATCH 08/48] SPARK-51016. fixed test failures after refactoring --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 9c365d7565a2..4b8762a4719e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -196,7 +196,7 @@ case class Alias(child: Expression, name: String)( override def toAttribute: Attribute = { if (resolved) { - AttributeReference(name, child.dataType, child.nullable)(exprId, qualifier, + AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifier, this.hasIndeterminism) } else { UnresolvedAttribute.quoted(name) @@ -367,7 +367,7 @@ case class AttributeReference( } override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: Nil + exprId :: qualifier :: Boolean.box(hasIndeterminism) :: Nil } /** Used to signal the column used to calculate an eventTime watermark (e.g. a#1-T{delayMs}) */ From 7878a601408650cef63241c1fa95c5e09e8719e5 Mon Sep 17 00:00:00 2001 From: ashahid Date: Mon, 3 Mar 2025 12:23:28 -0800 Subject: [PATCH 09/48] SPARK-51016. refcatored the tests to use inner joins instead of Left Outer to replicate the bug. --- .../catalyst/expressions/namedExpressions.scala | 1 - .../spark/scheduler/ShuffleMapStageTest.scala | 12 ++++++------ .../ProjectedOrderingAndPartitioningSuite.scala | 14 +++++++------- 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 4b8762a4719e..f6b8a96dca97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -112,7 +112,6 @@ abstract class Attribute extends LeafExpression with NamedExpression { override lazy val references: AttributeSet = AttributeSet(this) override def hasIndeterminism: Boolean = false - def withNullability(newNullability: Boolean): Attribute def withQualifier(newQualifier: Seq[String]): Attribute def withName(newName: String): Attribute diff --git a/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala b/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala index da3f619c4a2c..4733ae38d918 100644 --- a/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/scheduler/ShuffleMapStageTest.scala @@ -28,21 +28,21 @@ class ShuffleMapStageTest extends SharedSparkSession { test("SPARK-51016: ShuffleMapStage using indeterministic join keys should be INDETERMINATE") { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - val outerDf = spark.createDataset( + val leftDfBase = spark.createDataset( Seq((1L, "aa")))( Encoders.tuple(Encoders.scalaLong, Encoders.STRING)).toDF("pkLeftt", "strleft") - val innerDf = spark.createDataset( + val rightDf = spark.createDataset( Seq((1L, "11"), (2L, "22")))( Encoders.tuple(Encoders.scalaLong, Encoders.STRING)).toDF("pkRight", "strright") - val leftOuter = outerDf.select( + val leftDf = leftDfBase.select( col("strleft"), when(isnull(col("pkLeftt")), floor(rand() * Literal(10000000L)). cast(LongType)). otherwise(col("pkLeftt")).as("pkLeft")) - val outerjoin = leftOuter.hint("shuffle_hash"). - join(innerDf, col("pkLeft") === col("pkRight"), "left_outer") + val join = leftDf.hint("shuffle_hash"). + join(rightDf, col("pkLeft") === col("pkRight"), "inner") val shuffleStages: Array[ShuffleMapStage] = Array.ofDim(2) spark.sparkContext.addSparkListener(new SparkListener() { var i = 0 @@ -54,7 +54,7 @@ class ShuffleMapStageTest extends SharedSparkSession { } } }); - outerjoin.collect() + join.collect() assert(shuffleStages.filter(_.isIndeterminate).size == 1) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala index 395c03b81055..cf339559a169 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala @@ -216,24 +216,24 @@ class ProjectedOrderingAndPartitioningSuite test("SPARK-51016: ShuffleRDD using indeterministic join keys should be INDETERMINATE") { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - val outerDf = spark.createDataset( + val leftDfBase = spark.createDataset( Seq((1L, "aa"), (null, "aa"), (2L, "bb"), (null, "bb"), (3L, "cc"), (null, "cc")))( Encoders.tupleEncoder(Encoders.LONG, Encoders.STRING)).toDF("pkLeftt", "strleft") - val innerDf = spark.createDataset( + val rightDf = spark.createDataset( Seq((1L, "11"), (2L, "22"), (3L, "33")))( Encoders.tupleEncoder(Encoders.LONG, Encoders.STRING)).toDF("pkRight", "strright") - val leftOuter = outerDf.select( + val leftDf = leftDfBase.select( col("strleft"), when(isnull(col("pkLeftt")), floor(rand() * Literal(10000000L)). cast(LongType)). otherwise(col("pkLeftt")).as("pkLeft")) - val outerjoin = leftOuter.hint("shuffle_hash"). - join(innerDf, col("pkLeft") === col("pkRight"), "left_outer") + val join = leftDf.hint("shuffle_hash"). + join(rightDf, col("pkLeft") === col("pkRight"), "inner") - outerjoin.collect() - val finalPlan = outerjoin.queryExecution.executedPlan + join.collect() + val finalPlan = join.queryExecution.executedPlan val shuffleHJExec = finalPlan.children(0).asInstanceOf[ShuffledHashJoinExec] assert(shuffleHJExec.left.asInstanceOf[InputAdapter].execute().outputDeterministicLevel == DeterministicLevel.INDETERMINATE) From fc918af6beafad95f1160b184fb2f327cfe1aaf7 Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 4 Mar 2025 01:18:48 -0800 Subject: [PATCH 10/48] SPARK-51272: Reworked the test reproducing the race condition, by not relying on concurrent processing of DagEvent, instead relying on concurrent addition of event to the event loop queue --- .../spark/scheduler/DAGSchedulerSuite.scala | 97 ++++++++----------- 1 file changed, 38 insertions(+), 59 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 88175706bef8..08ed9803af0e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -51,7 +51,9 @@ import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockMan import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, Clock, LongAccumulator, SystemClock, ThreadUtils, Utils} import org.apache.spark.util.ArrayImplicits._ -class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) +class DAGSchedulerEventProcessLoopTester( + dagScheduler: DAGScheduler, + dagSchedulerInterceptorOpt: Option[DagSchedulerInterceptor] = None) extends DAGSchedulerEventProcessLoop(dagScheduler) { dagScheduler.setEventProcessLoop(this) @@ -64,12 +66,14 @@ class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) if (isProcessing) { // `DAGSchedulerEventProcessLoop` is guaranteed to process events sequentially. So we should // buffer events for sequent processing later instead of processing them recursively. + dagSchedulerInterceptorOpt.foreach(_.beforeAddingDagEventToQueue(event)) eventQueue += event } else { try { isProcessing = true // Forward event to `onReceive` directly to avoid processing event asynchronously. onReceive(event) + dagSchedulerInterceptorOpt.foreach(_.afterDirectProcessingOfDagEvent(event)) } catch { case NonFatal(e) => onError(e) } finally { @@ -81,15 +85,6 @@ class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) } } - def postInCurrentThread(event: DAGSchedulerEvent): Unit = { - try { - // Forward event to `onReceive` directly to avoid processing event asynchronously. - onReceive(event) - } catch { - case NonFatal(e) => onError(e) - } - } - override def onError(e: Throwable): Unit = { logError("Error in DAGSchedulerEventLoop: ", e) dagScheduler.stop() @@ -185,8 +180,8 @@ class DummyScheduledFuture( class DAGSchedulerSuiteDummyException extends Exception trait DagSchedulerInterceptor { - def interceptHandleTaskCompletion(event: CompletionEvent): Unit = {} - def interceptResubmitFailedStages(): Unit = {} + def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = {} + def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = {} } class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with TimeLimits { @@ -382,8 +377,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti env: SparkEnv, clock: Clock = new SystemClock(), shuffleMergeFinalize: Boolean = true, - shuffleMergeRegister: Boolean = true, - dagSchedulerInterceptorOpt: Option[DagSchedulerInterceptor] = None + shuffleMergeRegister: Boolean = true ) extends DAGScheduler( sc, taskScheduler, listenerBus, mapOutputTracker, blockManagerMaster, env, clock) { /** @@ -415,18 +409,12 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } override private[scheduler] def handleTaskCompletion(event: CompletionEvent): Unit = { - dagSchedulerInterceptorOpt.foreach(_.interceptHandleTaskCompletion(event)) super.handleTaskCompletion(event) runningTaskInfos.get(event.task.stageId).foreach{ partitions => partitions -= event.task.partitionId if (partitions.isEmpty) runningTaskInfos.remove(event.task.stageId) } } - - override private[scheduler] def resubmitFailedStages(): Unit = { - dagSchedulerInterceptorOpt.foreach(_.interceptResubmitFailedStages()) - super.resubmitFailedStages() - } } override def beforeEach(): Unit = { @@ -464,11 +452,11 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti sc.listenerBus, mapOutputTracker, blockManagerMaster, - sc.env, - dagSchedulerInterceptorOpt = Option(dagSchedulerInterceptor) + sc.env )) - dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler) + dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler, + Option(dagSchedulerInterceptor)) } override def afterEach(): Unit = { @@ -506,13 +494,6 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti dagEventProcessLoopTester.post(event) } - private def runEventInCurrentThread(event: DAGSchedulerEvent): Unit = { - // Ensure the initialization of various components - sc - dagEventProcessLoopTester.asInstanceOf[DAGSchedulerEventProcessLoopTester]. - postInCurrentThread(event) - } - /** * When we submit dummy Jobs, this is the compute function we supply. Except in a local test * below, we do not expect this function to ever be executed; instead, we will return results @@ -3222,15 +3203,15 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti if (scheduler != null) { this.afterEach() } - + val resubmitFailedStageTriggered = Array.fill[Boolean](1)(false) val monitor = new Object() - val resubmitFailedStageReached = Array.fill[Boolean](1)(false) this.dagSchedulerInterceptor = new DagSchedulerInterceptor { - override def interceptHandleTaskCompletion(event: CompletionEvent): Unit = { - event.reason match { - case Success if event.task.isInstanceOf[ResultTask[_, _]] => - assert(resubmitFailedStageReached(0)) + override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { + event match { + case ResubmitFailedStages => + runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) monitor.synchronized { + resubmitFailedStageTriggered(0) = true monitor.notify() } @@ -3238,11 +3219,21 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } } - override def interceptResubmitFailedStages(): Unit = { - monitor.synchronized { - resubmitFailedStageReached(0) = true - monitor.notify() - monitor.wait() + override def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = { + event match { + case CompletionEvent(_, reason, _, _, _, _) => + reason match { + case FetchFailed(_, _, _, _, _, _) => + monitor.synchronized { + if (!resubmitFailedStageTriggered(0)) { + monitor.wait() + } + } + + case _ => + } + + case _ => } } } @@ -3257,23 +3248,12 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti assert(activeJob.isDefined) // The result stage is still waiting for its 2 tasks to complete assert(resultStage.findMissingPartitions() == Seq.tabulate(numPartitions)(i => i)) - new Thread(() => { - runEventInCurrentThread( - makeCompletionEvent( - taskSets(2).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), - null)) - }).start() - - monitor.synchronized { - if (!resubmitFailedStageReached(0)) { - monitor.wait() - } - } - assert(resubmitFailedStageReached(0)) - new Thread(() => { - runEventInCurrentThread(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) - }).start() + + runEvent( + makeCompletionEvent( + taskSets(2).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), + null)) val shuffleStage1 = this.scheduler.shuffleIdToMapStage(shuffleId1) val shuffleStage2 = this.scheduler.shuffleIdToMapStage(shuffleId2) @@ -3295,7 +3275,6 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti Thread.sleep(500) keepGoing = resultStage.latestInfo.attemptNumber() != 1 } - assert(resultStage.latestInfo.numTasks == 2) } From d204a670280e0c987ce655d9be6fa7a4ea118955 Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 4 Mar 2025 11:33:41 -0800 Subject: [PATCH 11/48] SPARK-51272: formatting change --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 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 08ed9803af0e..aea20bc533ff 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -452,8 +452,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti sc.listenerBus, mapOutputTracker, blockManagerMaster, - sc.env - )) + sc.env)) dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler, Option(dagSchedulerInterceptor)) From 5a940a7dadfb700ce296f066c8441644e159c950 Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 4 Mar 2025 11:45:43 -0800 Subject: [PATCH 12/48] SPARK-51272: added some explanation to what test is doing to get the race condition exposed. --- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) 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 aea20bc533ff..55017d2b6ef2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3208,6 +3208,8 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { event match { case ResubmitFailedStages => + // Before the ResubmitFailedStages is added to the queue, add the successful + // partition task completion. runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) monitor.synchronized { resubmitFailedStageTriggered(0) = true @@ -3223,6 +3225,10 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti case CompletionEvent(_, reason, _, _, _, _) => reason match { case FetchFailed(_, _, _, _, _, _) => + // Do not allow this thread to exit, till the ResubmitFailedStages + // in callback is received. This is to ensure that this thread + // does not exit and process the ResubmitFailedStage event, before + // the queue gets successful partition task completion monitor.synchronized { if (!resubmitFailedStageTriggered(0)) { monitor.wait() @@ -3248,6 +3254,11 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // The result stage is still waiting for its 2 tasks to complete assert(resultStage.findMissingPartitions() == Seq.tabulate(numPartitions)(i => i)) + // The below event is going to initiate the retry of previous indeterminate stages, and also + // the retry of all result tasks. But before the "ResubmitFailedStages" event is added to the + // queue of Scheduler, a successful completion of the result partition task is added to the + // event queue. Due to scenario, the bug surfaces where instead of retry of all partitions + // of result tasks (2 tasks in total), only some (1 task) get retried runEvent( makeCompletionEvent( taskSets(2).tasks(0), @@ -3267,7 +3278,6 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti Thread.sleep(500) keepGoing = shuffleStage2.latestInfo.attemptNumber() != 1 } - completeShuffleMapStageSuccessfully(1, 1, numPartitions) keepGoing = true while (keepGoing) { From ddeeff807ad7c7534f9e2ddf74189f6b3efc8cca Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 4 Mar 2025 13:50:21 -0800 Subject: [PATCH 13/48] SPARK-51272: Given the insight by Attila that DagEventProcessing will always be in a single thread of EventLoop and that when ResubmitFailedStages event is submitted, though it is done by MessageScheduler ( a separate threadpool, but it is still doing post to the event loop's queue instead of direct onRecieve invocation, the read/write locks in stage are not needed --- .../apache/spark/scheduler/DAGScheduler.scala | 175 ++++++++---------- .../org/apache/spark/scheduler/Stage.scala | 61 +----- 2 files changed, 82 insertions(+), 154 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 a9efdd7ca1d0..8a97d74e7f1f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1898,9 +1898,8 @@ private[spark] class DAGScheduler( // Make sure the task's accumulators are updated before any other processing happens, so that // we can post a task end event before any jobs or stages are updated. The accumulators are // only updated in certain cases. - val (readLockTaken, isIndeterministicZombie) = event.reason match { + val isIndeterministicZombie = event.reason match { case Success => - stage.acquireStageReadLock() val isZombieIndeterminate = (task.stageAttemptId < stage.latestInfo.attemptNumber() && stage.isIndeterminate) || @@ -1920,29 +1919,14 @@ private[spark] class DAGScheduler( case _ => updateAccumulators(event) } } - (true, isZombieIndeterminate) + isZombieIndeterminate case _: ExceptionFailure | _: TaskKilled => updateAccumulators(event) - (false, false) + false - case _ => (false, false) + case _ => false } - - try { - handleTaskCompletionInOptionalReadLock(event, task, stageId, stage, isIndeterministicZombie) - } finally { - if (readLockTaken) { - stage.releaseStageReadLock() - } - } - } - - private def handleTaskCompletionInOptionalReadLock( - event: CompletionEvent, - task: Task[_], stageId: Int, - stage: Stage, - isIndeterministicZombie: Boolean): Unit = { if (trackingCacheVisibility) { // Update rdd blocks' visibility status. blockManagerMaster.updateRDDBlockVisibility( @@ -2010,7 +1994,7 @@ private[spark] class DAGScheduler( logInfo(log"Ignoring result from ${MDC(RESULT, rt)} because its job has finished") } - case smt: ShuffleMapTask if !isIndeterministicZombie => + case smt: ShuffleMapTask if !isIndeterministicZombie => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] // Ignore task completion for old attempt of indeterminate stage val ignoreIndeterminate = stage.isIndeterminate && @@ -2149,23 +2133,21 @@ private[spark] class DAGScheduler( failedStages += failedStage failedStages += mapStage if (noResubmitEnqueued) { - val writeLockedStages = mutable.Buffer.empty[Stage] - try { - // 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 UNORDERED, we are fine. The shuffle partitioner is - // guaranteed to be determinate, so the input data of the reducers will not change - // even if the map tasks are re-tried. - if (mapStage.isIndeterminate) { - // It's a little tricky to find all the succeeding stages of `mapStage`, 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 `mapStage`. 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 = HashSet[Stage](mapStage) + // 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 UNORDERED, we are fine. The shuffle partitioner is + // guaranteed to be determinate, so the input data of the reducers will not change + // even if the map tasks are re-tried. + if (mapStage.isIndeterminate) { + // It's a little tricky to find all the succeeding stages of `mapStage`, 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 `mapStage`. 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 = HashSet[Stage](mapStage) def collectStagesToRollback(stageChain: List[Stage]): Unit = { if (stagesToRollback.contains(stageChain.head)) { @@ -2177,33 +2159,30 @@ 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 indeterminacy by " + - "checkpointing the RDD before repartition and try again." - } - - activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) + 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 indeterminacy by " + + "checkpointing the RDD before repartition and try again." + } - // The stages will be rolled back after checking - val rollingBackStages = HashSet[Stage](mapStage) - stagesToRollback.foreach { - case mapStage: ShuffleMapStage => - if (mapStage.acquireStageWriteLock()) { - writeLockedStages += mapStage - } - val numMissingPartitions = mapStage.findMissingPartitions().length - if (numMissingPartitions < mapStage.numTasks) { - if (sc.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { - val reason = "A shuffle map stage with indeterminate output was failed " + - "and retried. However, Spark can only do this while using the new " + - "shuffle block fetching protocol. Please check the config " + - "'spark.shuffle.useOldFetchProtocol', see more detail in " + - "SPARK-27665 and SPARK-25341." - abortStage(mapStage, reason, None) - } else { - rollingBackStages += mapStage + activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) + + // The stages will be rolled back after checking + val rollingBackStages = HashSet[Stage](mapStage) + stagesToRollback.foreach { + case mapStage: ShuffleMapStage => + val numMissingPartitions = mapStage.findMissingPartitions().length + if (numMissingPartitions < mapStage.numTasks) { + if (sc.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + val reason = "A shuffle map stage with indeterminate output was failed " + + "and retried. However, Spark can only do this while using the new " + + "shuffle block fetching protocol. Please check the config " + + "'spark.shuffle.useOldFetchProtocol', see more detail in " + + "SPARK-27665 and SPARK-25341." + abortStage(mapStage, reason, None) + } else { + rollingBackStages += mapStage mapStage.markAttemptIdForAllPartitionsMissing( mapStage.latestInfo.attemptNumber()) } @@ -2212,48 +2191,42 @@ private[spark] class DAGScheduler( mapStage.latestInfo.attemptNumber()) } - case resultStage: ResultStage if resultStage.activeJob.isDefined => - if (resultStage.acquireStageWriteLock()) { - writeLockedStages += resultStage - } - val numMissingPartitions = resultStage.findMissingPartitions().length - if (numMissingPartitions < resultStage.numTasks) { - // TODO: support to rollback result tasks. - abortStage(resultStage, generateErrorMessage(resultStage), None) + case resultStage: ResultStage if resultStage.activeJob.isDefined => + val numMissingPartitions = resultStage.findMissingPartitions().length + if (numMissingPartitions < resultStage.numTasks) { + // TODO: support to rollback result tasks. + abortStage(resultStage, generateErrorMessage(resultStage), None) } else { resultStage.markAttemptIdForAllPartitionsMissing( resultStage.latestInfo.attemptNumber()) } - case _ => - } - logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + - log"we will roll back and rerun below stages which include itself and all its " + - log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") + case _ => } - - // 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 - // messages by checking whether a resubmit is already in the event queue for the - // failed stage. If there is already a resubmit enqueued for a different failed - // stage, that event would also be sufficient to handle the current failed stage, but - // producing a resubmit for each failed stage makes debugging and logging a little - // simpler while not producing an overwhelming number of scheduler events. - logInfo( - log"Resubmitting ${MDC(STAGE, mapStage)} " + - log"(${MDC(STAGE_NAME, mapStage.name)}) and ${MDC(FAILED_STAGE, failedStage)} " + - log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) due to fetch failure") - messageScheduler.schedule( - new Runnable { - override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) - }, - DAGScheduler.RESUBMIT_TIMEOUT, - TimeUnit.MILLISECONDS - ) - } finally { - writeLockedStages.foreach(_.releaseStageWriteLock()) + logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + + log"we will roll back and rerun below stages which include itself and all its " + + log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") } + + // 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 + // messages by checking whether a resubmit is already in the event queue for the + // failed stage. If there is already a resubmit enqueued for a different failed + // stage, that event would also be sufficient to handle the current failed stage, but + // producing a resubmit for each failed stage makes debugging and logging a little + // simpler while not producing an overwhelming number of scheduler events. + logInfo( + log"Resubmitting ${MDC(STAGE, mapStage)} " + + log"(${MDC(STAGE_NAME, mapStage.name)}) and ${MDC(FAILED_STAGE, failedStage)} " + + log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) due to fetch failure") + messageScheduler.schedule( + new Runnable { + override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) + }, + DAGScheduler.RESUBMIT_TIMEOUT, + TimeUnit.MILLISECONDS + ) } } @@ -2310,8 +2283,8 @@ private[spark] class DAGScheduler( log"and there is a more recent attempt for that stage (attempt " + log"${MDC(NUM_ATTEMPT, failedStage.latestInfo.attemptNumber())}) running") } else { - logInfo(log"Marking ${MDC(STAGE_ID, failedStage.id)} (${MDC(STAGE_NAME, failedStage.name)}) " + - log"as failed due to a barrier task failed.") + logInfo(log"Marking ${MDC(STAGE_ID, failedStage.id)} (${MDC(STAGE_NAME, failedStage.name)}) " + + log"as failed due to a barrier task failed.") val message = s"Stage failed because barrier task $task finished unsuccessfully.\n" + failure.toErrorString try { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 146b34ed8285..594ad6d55878 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import java.util.concurrent.locks.ReentrantReadWriteLock - import scala.collection.mutable.HashSet import org.apache.spark.executor.TaskMetrics @@ -68,9 +66,6 @@ private[scheduler] abstract class Stage( @volatile private var attemptIdAllPartitionsMissing: Int = -1 - private val stageReattemptLock = new ReentrantReadWriteLock() - private val stageReadLock = stageReattemptLock.readLock() - private val stageWriteLock = stageReattemptLock.writeLock() val numPartitions = rdd.partitions.length /** Set of jobs that this stage belongs to. */ @@ -108,21 +103,14 @@ private[scheduler] abstract class Stage( def makeNewStageAttempt( numPartitionsToCompute: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = { - val writeLockTaken = this.acquireStageWriteLock() - try { - val metrics = new TaskMetrics - metrics.register(rdd.sparkContext) - _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, - resourceProfileId = resourceProfileId) - nextAttemptId += 1 - // clear the entry in the allPartitionsAsMissing set - attemptIdAllPartitionsMissing = -1 - } finally { - if (writeLockTaken) { - this.releaseStageWriteLock() - } - } + val metrics = new TaskMetrics + metrics.register(rdd.sparkContext) + _latestInfo = StageInfo.fromStage( + this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, + resourceProfileId = resourceProfileId) + nextAttemptId += 1 + // clear the entry in the allPartitionsAsMissing set + attemptIdAllPartitionsMissing = -1 } /** Forward the nextAttemptId if skipped and get visited for the first time. */ @@ -154,37 +142,4 @@ private[scheduler] abstract class Stage( def markAttemptIdForAllPartitionsMissing(attemptId: Int): Unit = this.attemptIdAllPartitionsMissing = attemptId - - def acquireStageReadLock(): Unit = { - this.stageReadLock.lockInterruptibly() - val prevSet = Stage.threadHoldingReadLock.get() - Stage.threadHoldingReadLock.set(prevSet + this.id) - } - - def releaseStageReadLock(): Unit = { - val prevSet = Stage.threadHoldingReadLock.get() - Stage.threadHoldingReadLock.set(prevSet - this.id) - this.stageReadLock.unlock() - } - - def acquireStageWriteLock(): Boolean = { - if (Stage.threadHoldingReadLock.get().contains(this.id)) { - false - } else { - stageWriteLock.lockInterruptibly() - true - } - } - - def releaseStageWriteLock(): Unit = { - stageWriteLock.unlock() - } -} - -object Stage { - private val threadHoldingReadLock = new ThreadLocal[Set[Int]] { - override protected def initialValue(): Set[Int] = { - Set.empty[Int] - } - } } From c4e1fee6386b3136a3dd26bec41eb7996dc68b67 Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 4 Mar 2025 13:59:32 -0800 Subject: [PATCH 14/48] SPARK-51272: formatting changes. --- .../org/apache/spark/scheduler/DAGScheduler.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 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 8a97d74e7f1f..f78a16ca9667 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -2149,15 +2149,15 @@ private[spark] class DAGScheduler( // rollback, all its succeeding stages need to rollback to. val stagesToRollback = HashSet[Stage](mapStage) - 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) - } + 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) } } + } def generateErrorMessage(stage: Stage): String = { "A shuffle map stage with indeterminate output was failed and retried. " + From f84af25de268141cb9de837824215b20fd48212c Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 5 Mar 2025 17:40:35 -0800 Subject: [PATCH 15/48] initial changes --- resource-managers/yarn/pom.xml | 5 + .../spark/deploy/yarn/Spark51016Suite.scala | 202 ++++++++++++++++++ 2 files changed, 207 insertions(+) create mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 0629c24c56dc..d4bed6103491 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -37,6 +37,11 @@ spark-core_${scala.binary.version} ${project.version} + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + org.apache.spark spark-network-yarn_${scala.binary.version} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala new file mode 100644 index 000000000000..0346ec86bdda --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import com.google.common.io.Files +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.sql.{DataFrame, Encoders, QueryTest, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.deploy.yarn.YarnClusterDriver.{assert, be} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY, SUBMIT_DEPLOY_MODE} +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.sql.vectorized.Counter +import org.apache.spark.tags.ExtendedYarnTest +import org.apache.spark.util.{Utils, YarnContainerInfoHelper} +import org.scalatest.matchers.must.Matchers + +import java.io.File +import java.nio.charset.StandardCharsets +import scala.io.Source + +@ExtendedYarnTest +class Spark51016Suite extends BaseYarnClusterSuite { + + + test("run Spark in yarn-client mode with different configurations, ensuring redaction") { + testBasicYarnApp(true, + Map( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + "spark.task.maxFailures" -> "4", + "spark.network.timeout" -> "100000s", + "spark.shuffle.sort.bypassMergeThreshold" -> "1", + "spark.sql.files.maxPartitionNum" -> "2", + "spark.sql.files.minPartitionNum" -> "2", + DRIVER_MEMORY.key -> "512m", + EXECUTOR_CORES.key -> "1", + EXECUTOR_MEMORY.key -> "512m", + EXECUTOR_INSTANCES.key -> "2" + )) + } + + private def testBasicYarnApp(clientMode: Boolean, conf: Map[String, String] = Map()): Unit = { + val result = File.createTempFile("result", null, tempDir) + val finalState = runSpark(clientMode, mainClassName(Spark51016Suite.getClass), + appArgs = Seq(result.getAbsolutePath()), + extraConf = conf) + checkResult(finalState, result) + } +} + +private object Spark51016Suite extends Logging { + + object Counter { + var counter = 0 + var retVal = 12 + def getHash(): Int = this.synchronized { + counter += 1 + val x = retVal + if (counter % 6 == 0) { + retVal += 1 + } + x + } + } + + private def getOuterJoinDF(spark: SparkSession) = { + + import org.apache.spark.sql.functions.udf + val myudf = udf(() => Counter.getHash()).asNondeterministic() + spark.udf.register("myudf", myudf.asNondeterministic()) + + val leftOuter = spark.table("outer").select( + col("strleft"), when(isnull(col("pkLeftt")), myudf(). + cast(IntegerType)). + otherwise(col("pkLeftt")).as("pkLeft")) + + val innerRight = spark.table("inner") + + val outerjoin = leftOuter.hint("SHUFFLE_HASH"). + join(innerRight, col("pkLeft") === col("pkRight"), "left_outer") + outerjoin + + } + + def createBaseTables(spark: SparkSession): Unit = { + spark.sql("drop table if exists outer ") + spark.sql("drop table if exists inner ") + + val data = Seq((0, "aa"), (1, "aa"), (1, "aa"), (0, "aa"), (0, "aa"), (0, "aa"), + (null, "bb"), (null, "bb"), (null, "bb"), (null, "bb"), (null, "bb"), (null, "bb") + ) + val data1 = Seq((0, "bb"), (1, "bb")) + val outerDf = spark.createDataset(data)( + Encoders.tuple(Encoders.INT, Encoders.STRING)).toDF("pkLeftt", "strleft") + + this.logInfo("saving outer table") + outerDf.write.format("parquet").partitionBy("strleft").saveAsTable("outer") + + val innerDf = spark.createDataset(data1)( + Encoders.tuple(Encoders.INT, Encoders.STRING)).toDF("pkRight", "strright") + this.logInfo("saving inner table") + + innerDf.write.format("parquet").partitionBy("strright").saveAsTable("inner") + } + + def main(args: Array[String]): Unit = { + if (args.length != 1) { + // scalastyle:off println + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: Spark51016Suite [result file] + """.stripMargin) + // scalastyle:on println + System.exit(1) + } + + val spark = SparkSession.builder().appName("Spark51016Suite").getOrCreate() + + val status = new File(args(0)) + var result = "failure" + try { + createBaseTables(spark) + val outerjoin: DataFrame = getOuterJoinDF(spark) + + println("Initial data") + // outerjoin.show(100) + + val correctRows = outerjoin.collect() + TaskContext.unset() + for (i <- 0 until 100) { + try { + println("before query exec") + TaskContext.setFailResult() + val rowsAfterRetry = getOuterJoinDF(spark).collect() + TaskContext.unsetFailResult() + // import scala.jdk.CollectionConverters._ + // val temp = spark.createDataFrame(rowsAfterRetry.toSeq.asJava, outerjoin.schema) + + // temp.show(100) + if (correctRows.length != rowsAfterRetry.length) { + println(s"encounterted test failure incorrect query result. run index = $i ") + } + + assert(correctRows.length == rowsAfterRetry.length) + + val retriedResults = rowsAfterRetry.toBuffer + correctRows.foreach(r => { + val index = retriedResults.indexWhere(x => + + r.getString(0) == x.getString(0) && + ( + (r.getInt(1) < 2 && r.getInt(1) == x.getInt(1) && r.getInt(2) == x.getInt(2) && + r.getString(3) == x.getString(3)) + || + (r.isNullAt(2) && r.isNullAt(3) && x.isNullAt(3) + && x.isNullAt(2)) + + )) + assert(index >= 0) + retriedResults.remove(index) + } + ) + assert(retriedResults.isEmpty) + println(s"found successful query exec on iter index = $i") + } catch { + case se: SparkException if se.getMessage.contains("Please eliminate the" + + " indeterminacy by checkpointing the RDD before repartition and try again") => + println(s"correctly encountered exception on iter index = $i") + // OK expected + } + } + + } finally { + Files.asCharSink(status, StandardCharsets.UTF_8).write(result) + sc.stop() + } + } + +} \ No newline at end of file From ac963bf111b3bf0803137d57b0f666624c9b6035 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 5 Mar 2025 18:33:38 -0800 Subject: [PATCH 16/48] SPARK-51272. incorporating review feedback --- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 55017d2b6ef2..4667d8646165 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3202,7 +3202,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti if (scheduler != null) { this.afterEach() } - val resubmitFailedStageTriggered = Array.fill[Boolean](1)(false) + var resubmitFailedStageTriggered = false val monitor = new Object() this.dagSchedulerInterceptor = new DagSchedulerInterceptor { override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { @@ -3212,7 +3212,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // partition task completion. runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) monitor.synchronized { - resubmitFailedStageTriggered(0) = true + resubmitFailedStageTriggered = true monitor.notify() } @@ -3230,7 +3230,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // does not exit and process the ResubmitFailedStage event, before // the queue gets successful partition task completion monitor.synchronized { - if (!resubmitFailedStageTriggered(0)) { + if (!resubmitFailedStageTriggered) { monitor.wait() } } From d6523194705087c24928d4868b9f04b77bf4adc5 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 5 Mar 2025 18:38:40 -0800 Subject: [PATCH 17/48] SPARK-51272. incorporating review feedback. removing explict invocation of beforeEach and afterEach in test --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 5 ----- 1 file changed, 5 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 4667d8646165..9301c2abe843 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3199,9 +3199,6 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti test("SPARK-51272: retry all the succeeding stages when the map stage is indeterminate with" + " concurrent tasks completion") { - if (scheduler != null) { - this.afterEach() - } var resubmitFailedStageTriggered = false val monitor = new Object() this.dagSchedulerInterceptor = new DagSchedulerInterceptor { @@ -3243,8 +3240,6 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } } - this.beforeEach() - val numPartitions = 2 val (shuffleId1, shuffleId2) = constructTwoIndeterminateStage() completeShuffleMapStageSuccessfully(shuffleId2, 0, numPartitions) From d8b40790d091b6e9bdf577876d351fa1777fdb3c Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 6 Mar 2025 13:23:15 -0800 Subject: [PATCH 18/48] initial changes --- .../spark/deploy/yarn/Spark51016Suite.scala | 59 +++++++++++-------- 1 file changed, 33 insertions(+), 26 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index 0346ec86bdda..ce434d304981 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -17,31 +17,24 @@ package org.apache.spark.deploy.yarn +import java.io.File +import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.spark.{SparkException, TaskContext} -import org.apache.spark.sql.{DataFrame, Encoders, QueryTest, SparkSession} +import org.apache.spark.{SparkContext, SparkException, TaskContext} +import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.IntegerType -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.deploy.yarn.YarnClusterDriver.{assert, be} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY, SUBMIT_DEPLOY_MODE} -import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.sql.vectorized.Counter +import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} import org.apache.spark.tags.ExtendedYarnTest -import org.apache.spark.util.{Utils, YarnContainerInfoHelper} -import org.scalatest.matchers.must.Matchers - -import java.io.File -import java.nio.charset.StandardCharsets -import scala.io.Source @ExtendedYarnTest class Spark51016Suite extends BaseYarnClusterSuite { + override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() + test("run Spark in yarn-client mode with different configurations, ensuring redaction") { testBasicYarnApp(true, @@ -54,9 +47,11 @@ class Spark51016Suite extends BaseYarnClusterSuite { "spark.sql.files.maxPartitionNum" -> "2", "spark.sql.files.minPartitionNum" -> "2", DRIVER_MEMORY.key -> "512m", - EXECUTOR_CORES.key -> "1", + EXECUTOR_CORES.key -> "2", EXECUTOR_MEMORY.key -> "512m", - EXECUTOR_INSTANCES.key -> "2" + EXECUTOR_INSTANCES.key -> "2", + "spark.ui.port" -> "4040", + "spark.ui.enabled" -> "true" )) } @@ -107,10 +102,23 @@ private object Spark51016Suite extends Logging { spark.sql("drop table if exists outer ") spark.sql("drop table if exists inner ") - val data = Seq((0, "aa"), (1, "aa"), (1, "aa"), (0, "aa"), (0, "aa"), (0, "aa"), - (null, "bb"), (null, "bb"), (null, "bb"), (null, "bb"), (null, "bb"), (null, "bb") + val data = Seq( + (java.lang.Integer.valueOf(0), "aa"), + (java.lang.Integer.valueOf(1), "aa"), + (java.lang.Integer.valueOf(1), "aa"), + (java.lang.Integer.valueOf(0), "aa"), + (java.lang.Integer.valueOf(0), "aa"), + (java.lang.Integer.valueOf(0), "aa"), + (null, "bb"), + (null, "bb"), + (null, "bb"), + (null, "bb"), + (null, "bb"), + (null, "bb") ) - val data1 = Seq((0, "bb"), (1, "bb")) + val data1 = Seq( + (java.lang.Integer.valueOf(0), "bb"), + (java.lang.Integer.valueOf(1), "bb")) val outerDf = spark.createDataset(data)( Encoders.tuple(Encoders.INT, Encoders.STRING)).toDF("pkLeftt", "strleft") @@ -138,6 +146,7 @@ private object Spark51016Suite extends Logging { } val spark = SparkSession.builder().appName("Spark51016Suite").getOrCreate() + val sc = SparkContext.getOrCreate() val status = new File(args(0)) var result = "failure" @@ -145,7 +154,7 @@ private object Spark51016Suite extends Logging { createBaseTables(spark) val outerjoin: DataFrame = getOuterJoinDF(spark) - println("Initial data") + // println("Initial data") // outerjoin.show(100) val correctRows = outerjoin.collect() @@ -153,13 +162,10 @@ private object Spark51016Suite extends Logging { for (i <- 0 until 100) { try { println("before query exec") - TaskContext.setFailResult() + // TaskContext.setFailResult() val rowsAfterRetry = getOuterJoinDF(spark).collect() - TaskContext.unsetFailResult() - // import scala.jdk.CollectionConverters._ - // val temp = spark.createDataFrame(rowsAfterRetry.toSeq.asJava, outerjoin.schema) + // TaskContext.unsetFailResult() - // temp.show(100) if (correctRows.length != rowsAfterRetry.length) { println(s"encounterted test failure incorrect query result. run index = $i ") } @@ -192,7 +198,8 @@ private object Spark51016Suite extends Logging { // OK expected } } - + Thread.sleep(1000000) + result = "success" } finally { Files.asCharSink(status, StandardCharsets.UTF_8).write(result) sc.stop() From a0c700c215b58b9f5dc78c93319114688250e366 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 6 Mar 2025 13:30:41 -0800 Subject: [PATCH 19/48] SPARK-51272. rectified a comment and renamed the test --- core/src/main/scala/org/apache/spark/scheduler/Stage.scala | 2 +- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 594ad6d55878..2ff110e43bc7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -109,7 +109,7 @@ private[scheduler] abstract class Stage( this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, resourceProfileId = resourceProfileId) nextAttemptId += 1 - // clear the entry in the allPartitionsAsMissing set + // clear the attemptId set in the attemptIdAllPartitionsMissing attemptIdAllPartitionsMissing = -1 } 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 9301c2abe843..4a68e5be74c5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3197,8 +3197,8 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti "Spark can only do this while using the new shuffle block fetching protocol")) } - test("SPARK-51272: retry all the succeeding stages when the map stage is indeterminate with" + - " concurrent tasks completion") { + test("SPARK-51272: retry all the partitions of result stage, if the first result task" + + " has failed and ShuffleMap stage is inDeterminate") { var resubmitFailedStageTriggered = false val monitor = new Object() this.dagSchedulerInterceptor = new DagSchedulerInterceptor { From a930c9d3cdf8719af9f7156f9ad4bacbf85d5648 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 6 Mar 2025 19:52:44 -0800 Subject: [PATCH 20/48] initial changes --- .../deploy/yarn/BaseYarnClusterSuite.scala | 6 +- .../spark/deploy/yarn/Spark51016Suite.scala | 92 ++++++++++++++++--- 2 files changed, 85 insertions(+), 13 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 3bf6a6e84a88..2409f4a07a3a 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -181,10 +181,12 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { launcher.setMainClass(klass) launcher.setAppResource(fakeSparkJar.getAbsolutePath()) } + + val numExecsOpt = extraConf.get(EXECUTOR_INSTANCES.key) launcher.setSparkHome(sys.props("spark.test.home")) .setMaster("yarn") .setDeployMode(deployMode) - .setConf(EXECUTOR_INSTANCES.key, "1") + .setConf(EXECUTOR_INSTANCES.key, numExecsOpt.getOrElse("1")) .setConf(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS, s"-Djava.net.preferIPv6Addresses=${Utils.preferIPv6}") .setPropertiesFile(propsFile) @@ -210,7 +212,7 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { val handle = launcher.startApplication() try { - eventually(timeout(3.minutes), interval(1.second)) { + eventually(timeout(30.minutes), interval(10.second)) { assert(handle.getState().isFinal()) } } finally { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index ce434d304981..bfe2be0d79c2 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -17,18 +17,24 @@ package org.apache.spark.deploy.yarn -import java.io.File -import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.spark.{SparkContext, SparkException, TaskContext} -import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageSubmitted} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} +import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} import org.apache.spark.tags.ExtendedYarnTest +import org.apache.spark.{SparkContext, SparkException} +import org.scalatest.concurrent.Eventually._ + +import java.io.File +import java.nio.charset.StandardCharsets +import scala.concurrent.duration._ + + @ExtendedYarnTest class Spark51016Suite extends BaseYarnClusterSuite { @@ -145,7 +151,11 @@ private object Spark51016Suite extends Logging { System.exit(1) } - val spark = SparkSession.builder().appName("Spark51016Suite").getOrCreate() + val spark = SparkSession + .builder() + .appName("Spark51016Suite") + .config("spark.extraListeners", classOf[JobListener].getName) + .getOrCreate() val sc = SparkContext.getOrCreate() val status = new File(args(0)) @@ -158,13 +168,14 @@ private object Spark51016Suite extends Logging { // outerjoin.show(100) val correctRows = outerjoin.collect() - TaskContext.unset() + JobListener.inKillMode = true for (i <- 0 until 100) { try { + eventually(timeout(3.minutes), interval(100.milliseconds)) { + assert(sc.getExecutorIds().size == 2) + } println("before query exec") - // TaskContext.setFailResult() val rowsAfterRetry = getOuterJoinDF(spark).collect() - // TaskContext.unsetFailResult() if (correctRows.length != rowsAfterRetry.length) { println(s"encounterted test failure incorrect query result. run index = $i ") @@ -198,12 +209,71 @@ private object Spark51016Suite extends Logging { // OK expected } } - Thread.sleep(1000000) + result = "success" } finally { Files.asCharSink(status, StandardCharsets.UTF_8).write(result) sc.stop() } } +} + +object PIDGetter { + def getExecutorIds(): Seq[Int] = { + import scala.sys.process._ + val output = Seq("ps", "-ef").#|(Seq("grep", "java")).#|(Seq("grep", "executor-id ")).lazyLines + // println(output.mkString("\n\n")) + if (output.nonEmpty && output.size > 3) { + val execPidsStr = Seq(output(1).trim, output(3).trim) + val pids = execPidsStr.map(str => str.split(" ")(1).toInt) + pids + } else { + Seq.empty + } + + } + + def killExecutor(pid: Int): Unit = { + import scala.sys.process._ + Seq("kill", "-9", pid.toString).! + + } + + def main(args: Array[String]): Unit = { + getExecutorIds() + } +} + +private[spark] class JobListener extends SparkListener { + private var kill: Boolean = false + private var count: Int = 0 + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + if (JobListener.inKillMode) { + kill = true + count += 1 + } + } + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + if (stageSubmitted.stageInfo.attemptNumber() == 0 && + stageSubmitted.stageInfo.shuffleDepId.nonEmpty && kill) { + kill = false + val killThread = new Thread(new Runnable() { + override def run(): Unit = { + val execids = PIDGetter.getExecutorIds() + if (execids.size == 2) { + val pidToKill = execids(count % 2) + PIDGetter.killExecutor(pidToKill) + } + } + }) + killThread.start() + killThread.join() + } + } + +} +object JobListener { + @volatile + var inKillMode: Boolean = false } \ No newline at end of file From 9b153a37f3dbc10f11b3e83151d0814066280fb0 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 6 Mar 2025 22:00:59 -0800 Subject: [PATCH 21/48] SPARK-51272. incorporated review feedback to refactor test --- .../spark/scheduler/DAGSchedulerSuite.scala | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 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 4a68e5be74c5..1d50bc199f7a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3259,27 +3259,23 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti taskSets(2).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), null)) - val shuffleStage1 = this.scheduler.shuffleIdToMapStage(shuffleId1) val shuffleStage2 = this.scheduler.shuffleIdToMapStage(shuffleId2) - var keepGoing = true - while (keepGoing) { - Thread.sleep(500) - keepGoing = shuffleStage1.latestInfo.attemptNumber() != 1 - } completeShuffleMapStageSuccessfully(0, 1, numPartitions) - keepGoing = true - while (keepGoing) { - Thread.sleep(500) - keepGoing = shuffleStage2.latestInfo.attemptNumber() != 1 + import org.scalatest.concurrent.Eventually._ + import org.scalatest.matchers.should.Matchers._ + import org.scalatest.time.SpanSugar._ + eventually(timeout(3.minutes), interval(500.milliseconds)) { + shuffleStage1.latestInfo.attemptNumber() should equal(1) + } + eventually(timeout(3.minutes), interval(500.milliseconds)) { + shuffleStage2.latestInfo.attemptNumber() should equal(1) } completeShuffleMapStageSuccessfully(1, 1, numPartitions) - keepGoing = true - while (keepGoing) { - Thread.sleep(500) - keepGoing = resultStage.latestInfo.attemptNumber() != 1 + eventually(timeout(3.minutes), interval(500.milliseconds)) { + resultStage.latestInfo.attemptNumber() should equal(1) } - assert(resultStage.latestInfo.numTasks == 2) + org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == 2) } test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { From 70b7910094c30520fbc8779b3dbadb56908e53f0 Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 7 Mar 2025 10:38:00 -0800 Subject: [PATCH 22/48] SPARK-51272. incorporated review feedback to refactor test --- .../spark/scheduler/DAGSchedulerSuite.scala | 18 +++++------------- 1 file changed, 5 insertions(+), 13 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 1d50bc199f7a..afee74b16417 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3199,19 +3199,15 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti test("SPARK-51272: retry all the partitions of result stage, if the first result task" + " has failed and ShuffleMap stage is inDeterminate") { - var resubmitFailedStageTriggered = false - val monitor = new Object() + val latch = new CountDownLatch(1) this.dagSchedulerInterceptor = new DagSchedulerInterceptor { override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { event match { case ResubmitFailedStages => - // Before the ResubmitFailedStages is added to the queue, add the successful - // partition task completion. + // Before the ResubmitFailedStages is added to the queue, add the successful + // partition task completion. runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) - monitor.synchronized { - resubmitFailedStageTriggered = true - monitor.notify() - } + latch.countDown() case _ => } @@ -3226,11 +3222,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // in callback is received. This is to ensure that this thread // does not exit and process the ResubmitFailedStage event, before // the queue gets successful partition task completion - monitor.synchronized { - if (!resubmitFailedStageTriggered) { - monitor.wait() - } - } + latch.await(50, TimeUnit.SECONDS) case _ => } From a5ddc9e2f98d7b34ba51e9b078c2cc9e0c6870b5 Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 7 Mar 2025 15:22:34 -0800 Subject: [PATCH 23/48] initial changes --- .../spark/deploy/yarn/Spark51016Suite.scala | 34 +++++++++++++++---- 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index bfe2be0d79c2..150b163124ad 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -21,7 +21,7 @@ import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageSubmitted} +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType @@ -47,17 +47,18 @@ class Spark51016Suite extends BaseYarnClusterSuite { Map( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "2", - "spark.task.maxFailures" -> "4", + "spark.task.maxFailures" -> "8", "spark.network.timeout" -> "100000s", "spark.shuffle.sort.bypassMergeThreshold" -> "1", "spark.sql.files.maxPartitionNum" -> "2", "spark.sql.files.minPartitionNum" -> "2", DRIVER_MEMORY.key -> "512m", - EXECUTOR_CORES.key -> "2", + EXECUTOR_CORES.key -> "1", EXECUTOR_MEMORY.key -> "512m", EXECUTOR_INSTANCES.key -> "2", "spark.ui.port" -> "4040", - "spark.ui.enabled" -> "true" + "spark.ui.enabled" -> "true", + "spark.yarn.max.executor.failures" -> "100000" )) } @@ -224,7 +225,7 @@ object PIDGetter { val output = Seq("ps", "-ef").#|(Seq("grep", "java")).#|(Seq("grep", "executor-id ")).lazyLines // println(output.mkString("\n\n")) if (output.nonEmpty && output.size > 3) { - val execPidsStr = Seq(output(1).trim, output(3).trim) + val execPidsStr = Seq(output(0).trim, output(2).trim) val pids = execPidsStr.map(str => str.split(" ")(1).toInt) pids } else { @@ -253,8 +254,27 @@ private[spark] class JobListener extends SparkListener { count += 1 } } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + if (stageCompleted.stageInfo.shuffleDepId.nonEmpty && kill + && stageCompleted.stageInfo.shuffleDepId.get % 2 == 0 ) { + kill = false + val killThread = new Thread(new Runnable() { + override def run(): Unit = { + val execids = PIDGetter.getExecutorIds() + if (execids.size == 2) { + val pidToKill = execids(count % 2) + PIDGetter.killExecutor(pidToKill) + } + } + }) + killThread.start() + killThread.join() + } + } + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { - if (stageSubmitted.stageInfo.attemptNumber() == 0 && + /* if (stageSubmitted.stageInfo.attemptNumber() == 0 && stageSubmitted.stageInfo.shuffleDepId.nonEmpty && kill) { kill = false val killThread = new Thread(new Runnable() { @@ -268,7 +288,7 @@ private[spark] class JobListener extends SparkListener { }) killThread.start() killThread.join() - } + } */ } } From 001258abf6894b0f17d920c7158b31727fbb1989 Mon Sep 17 00:00:00 2001 From: ashahid Date: Mon, 10 Mar 2025 13:35:29 -0700 Subject: [PATCH 24/48] initial changes --- .../deploy/yarn/BaseYarnClusterSuite.scala | 6 +- .../spark/deploy/yarn/Spark51016Suite.scala | 144 ++++++------------ 2 files changed, 52 insertions(+), 98 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 2409f4a07a3a..59ec771316c0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -167,7 +167,9 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { extraJars: Seq[String] = Nil, extraConf: Map[String, String] = Map(), extraEnv: Map[String, String] = Map(), - outFile: Option[File] = None): SparkAppHandle.State = { + outFile: Option[File] = None, + testTimeOut: Int = 3, // minutes + timeOutIntervalCheck: Int = 1 /* seconds */): SparkAppHandle.State = { val deployMode = if (clientMode) "client" else "cluster" val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf) val env = Map( @@ -212,7 +214,7 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { val handle = launcher.startApplication() try { - eventually(timeout(30.minutes), interval(10.second)) { + eventually(timeout(testTimeOut.minutes), interval(timeOutIntervalCheck.second)) { assert(handle.getState().isFinal()) } } finally { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index 150b163124ad..dc0194c1157d 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -17,33 +17,30 @@ package org.apache.spark.deploy.yarn +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.concurrent.duration._ + import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.scalatest.concurrent.Eventually._ + import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted} +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageSubmitted} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} import org.apache.spark.tags.ExtendedYarnTest import org.apache.spark.{SparkContext, SparkException} -import org.scalatest.concurrent.Eventually._ - -import java.io.File -import java.nio.charset.StandardCharsets -import scala.concurrent.duration._ - - @ExtendedYarnTest class Spark51016Suite extends BaseYarnClusterSuite { - override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() - - - test("run Spark in yarn-client mode with different configurations, ensuring redaction") { - testBasicYarnApp(true, + test("bug SPARK-51016 and SPARK-51272: Indeterminate stage retry giving wrong results") { + testBasicYarnApp( Map( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "2", @@ -62,11 +59,15 @@ class Spark51016Suite extends BaseYarnClusterSuite { )) } - private def testBasicYarnApp(clientMode: Boolean, conf: Map[String, String] = Map()): Unit = { + private def testBasicYarnApp(conf: Map[String, String] = Map()): Unit = { val result = File.createTempFile("result", null, tempDir) - val finalState = runSpark(clientMode, mainClassName(Spark51016Suite.getClass), - appArgs = Seq(result.getAbsolutePath()), - extraConf = conf) + val finalState = runSpark( + clientMode = true, + mainClassName(Spark51016Suite.getClass), + appArgs = Seq(result.getAbsolutePath), + extraConf = conf, + testTimeOut = 30, + timeOutIntervalCheck = 30) checkResult(finalState, result) } } @@ -76,6 +77,7 @@ private object Spark51016Suite extends Logging { object Counter { var counter = 0 var retVal = 12 + def getHash(): Int = this.synchronized { counter += 1 val x = retVal @@ -87,7 +89,6 @@ private object Spark51016Suite extends Logging { } private def getOuterJoinDF(spark: SparkSession) = { - import org.apache.spark.sql.functions.udf val myudf = udf(() => Counter.getHash()).asNondeterministic() spark.udf.register("myudf", myudf.asNondeterministic()) @@ -102,13 +103,11 @@ private object Spark51016Suite extends Logging { val outerjoin = leftOuter.hint("SHUFFLE_HASH"). join(innerRight, col("pkLeft") === col("pkRight"), "left_outer") outerjoin - } def createBaseTables(spark: SparkSession): Unit = { spark.sql("drop table if exists outer ") spark.sql("drop table if exists inner ") - val data = Seq( (java.lang.Integer.valueOf(0), "aa"), (java.lang.Integer.valueOf(1), "aa"), @@ -128,30 +127,15 @@ private object Spark51016Suite extends Logging { (java.lang.Integer.valueOf(1), "bb")) val outerDf = spark.createDataset(data)( Encoders.tuple(Encoders.INT, Encoders.STRING)).toDF("pkLeftt", "strleft") - this.logInfo("saving outer table") outerDf.write.format("parquet").partitionBy("strleft").saveAsTable("outer") - val innerDf = spark.createDataset(data1)( Encoders.tuple(Encoders.INT, Encoders.STRING)).toDF("pkRight", "strright") this.logInfo("saving inner table") - innerDf.write.format("parquet").partitionBy("strright").saveAsTable("inner") } def main(args: Array[String]): Unit = { - if (args.length != 1) { - // scalastyle:off println - System.err.println( - s""" - |Invalid command line: ${args.mkString(" ")} - | - |Usage: Spark51016Suite [result file] - """.stripMargin) - // scalastyle:on println - System.exit(1) - } - val spark = SparkSession .builder() .appName("Spark51016Suite") @@ -164,10 +148,6 @@ private object Spark51016Suite extends Logging { try { createBaseTables(spark) val outerjoin: DataFrame = getOuterJoinDF(spark) - - // println("Initial data") - // outerjoin.show(100) - val correctRows = outerjoin.collect() JobListener.inKillMode = true for (i <- 0 until 100) { @@ -175,15 +155,11 @@ private object Spark51016Suite extends Logging { eventually(timeout(3.minutes), interval(100.milliseconds)) { assert(sc.getExecutorIds().size == 2) } - println("before query exec") val rowsAfterRetry = getOuterJoinDF(spark).collect() - if (correctRows.length != rowsAfterRetry.length) { - println(s"encounterted test failure incorrect query result. run index = $i ") + logInfo(s"encounterted test failure incorrect query result. run index = $i ") } - assert(correctRows.length == rowsAfterRetry.length) - val retriedResults = rowsAfterRetry.toBuffer correctRows.foreach(r => { val index = retriedResults.indexWhere(x => @@ -199,18 +175,17 @@ private object Spark51016Suite extends Logging { )) assert(index >= 0) retriedResults.remove(index) - } - ) + }) assert(retriedResults.isEmpty) - println(s"found successful query exec on iter index = $i") + logInfo(s"found successful query exec on iter index = $i") + Thread.sleep(500) } catch { case se: SparkException if se.getMessage.contains("Please eliminate the" + " indeterminacy by checkpointing the RDD before repartition and try again") => - println(s"correctly encountered exception on iter index = $i") + logInfo(s"correctly encountered exception on iter index = $i") // OK expected } } - result = "success" } finally { Files.asCharSink(status, StandardCharsets.UTF_8).write(result) @@ -219,19 +194,19 @@ private object Spark51016Suite extends Logging { } } -object PIDGetter { - def getExecutorIds(): Seq[Int] = { +object PIDGetter extends Logging { + def getExecutorPIds: Seq[Int] = { import scala.sys.process._ val output = Seq("ps", "-ef").#|(Seq("grep", "java")).#|(Seq("grep", "executor-id ")).lazyLines - // println(output.mkString("\n\n")) - if (output.nonEmpty && output.size > 3) { - val execPidsStr = Seq(output(0).trim, output(2).trim) - val pids = execPidsStr.map(str => str.split(" ")(1).toInt) - pids + logInfo(s"pids obtained = ${output.mkString("\n")} ") + if (output.nonEmpty && output.size == 4) { + val execPidsStr = output.map(_.trim).filter(_.endsWith("--resourceProfileId 0")) + logInfo(s"filtered Pid String obtained = ${execPidsStr.mkString("\n")} ") + val pids = execPidsStr.map(str => str.split(" ")(1).toInt).sorted + Seq(pids.head, pids(1)) } else { Seq.empty } - } def killExecutor(pid: Int): Unit = { @@ -241,59 +216,36 @@ object PIDGetter { } def main(args: Array[String]): Unit = { - getExecutorIds() + getExecutorPIds } } -private[spark] class JobListener extends SparkListener { - private var kill: Boolean = false +private[spark] class JobListener extends SparkListener with Logging { private var count: Int = 0 + @volatile + private var pidToKill: Option[Int] = None + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { if (JobListener.inKillMode) { - kill = true + val execids = PIDGetter.getExecutorPIds + assert(execids.size == 2) + pidToKill = Option(execids(count % 2)) + logInfo("Pid to kill = " + pidToKill) count += 1 } } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { - if (stageCompleted.stageInfo.shuffleDepId.nonEmpty && kill - && stageCompleted.stageInfo.shuffleDepId.get % 2 == 0 ) { - kill = false - val killThread = new Thread(new Runnable() { - override def run(): Unit = { - val execids = PIDGetter.getExecutorIds() - if (execids.size == 2) { - val pidToKill = execids(count % 2) - PIDGetter.killExecutor(pidToKill) - } - } - }) - killThread.start() - killThread.join() - } - } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { - /* if (stageSubmitted.stageInfo.attemptNumber() == 0 && - stageSubmitted.stageInfo.shuffleDepId.nonEmpty && kill) { - kill = false - val killThread = new Thread(new Runnable() { - override def run(): Unit = { - val execids = PIDGetter.getExecutorIds() - if (execids.size == 2) { - val pidToKill = execids(count % 2) - PIDGetter.killExecutor(pidToKill) - } - } - }) - killThread.start() - killThread.join() - } */ + if (stageSubmitted.stageInfo.shuffleDepId.isEmpty && pidToKill.nonEmpty) { + val pid = pidToKill.get + pidToKill = None + logInfo(s"killing executor for pid = $pid") + PIDGetter.killExecutor(pid) + } } - } object JobListener { @volatile var inKillMode: Boolean = false -} \ No newline at end of file +} From a819f4f72c38ce078df589b23de7d76f87c6fc31 Mon Sep 17 00:00:00 2001 From: ashahid Date: Mon, 10 Mar 2025 14:44:16 -0700 Subject: [PATCH 25/48] initial changes --- .../scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index dc0194c1157d..a3ae8c4f363a 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -159,7 +159,9 @@ private object Spark51016Suite extends Logging { if (correctRows.length != rowsAfterRetry.length) { logInfo(s"encounterted test failure incorrect query result. run index = $i ") } - assert(correctRows.length == rowsAfterRetry.length) + assert(correctRows.length == rowsAfterRetry.length, + s"correct rows length = ${correctRows.length}," + + s" retry rows length = ${rowsAfterRetry.length}") val retriedResults = rowsAfterRetry.toBuffer correctRows.foreach(r => { val index = retriedResults.indexWhere(x => From d8eb1c30878e1027f5e10ce3751afa2707fc69d0 Mon Sep 17 00:00:00 2001 From: ashahid Date: Mon, 10 Mar 2025 14:59:08 -0700 Subject: [PATCH 26/48] SPARK-51272. implemented review feedback. refactored code. no longer passing attempt id as parameter --- .../apache/spark/scheduler/DAGScheduler.scala | 19 ++++++++----------- .../apache/spark/scheduler/ResultStage.scala | 2 +- .../spark/scheduler/ShuffleMapStage.scala | 2 +- .../org/apache/spark/scheduler/Stage.scala | 8 ++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 14 ++------------ 5 files changed, 16 insertions(+), 29 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 f78a16ca9667..48cc239a86ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1903,7 +1903,7 @@ private[spark] class DAGScheduler( val isZombieIndeterminate = (task.stageAttemptId < stage.latestInfo.attemptNumber() && stage.isIndeterminate) || - stage.treatAllPartitionsMissing(task.stageAttemptId) + stage.areAllPartitionsMissing(task.stageAttemptId) if (!isZombieIndeterminate) { task match { case rt: ResultTask[_, _] => @@ -2183,23 +2183,20 @@ private[spark] class DAGScheduler( abortStage(mapStage, reason, None) } else { rollingBackStages += mapStage - mapStage.markAttemptIdForAllPartitionsMissing( - mapStage.latestInfo.attemptNumber()) - } - } else { - mapStage.markAttemptIdForAllPartitionsMissing( - mapStage.latestInfo.attemptNumber()) + mapStage.markAllPartitionsMissing() } + } else { + mapStage.markAllPartitionsMissing() + } case resultStage: ResultStage if resultStage.activeJob.isDefined => val numMissingPartitions = resultStage.findMissingPartitions().length if (numMissingPartitions < resultStage.numTasks) { // TODO: support to rollback result tasks. abortStage(resultStage, generateErrorMessage(resultStage), None) - } else { - resultStage.markAttemptIdForAllPartitionsMissing( - resultStage.latestInfo.attemptNumber()) - } + } else { + resultStage.markAllPartitionsMissing() + } case _ => } 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 97130ddd7455..a364f5ad21e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -62,7 +62,7 @@ private[spark] class ResultStage( override def findMissingPartitions(): Seq[Int] = { val job = activeJob.get val allPartitions = (0 until job.numPartitions) - if (this.treatAllPartitionsMissing(this.latestInfo.attemptNumber())) { + if (this.areAllPartitionsMissing(this.latestInfo.attemptNumber())) { allPartitions } else { allPartitions.filter(id => !job.finished(id)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 57d53b89d397..5bbbf34d4453 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -90,7 +90,7 @@ private[spark] class ShuffleMapStage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ override def findMissingPartitions(): Seq[Int] = { - if (this.treatAllPartitionsMissing(this.latestInfo.attemptNumber())) { + if (this.areAllPartitionsMissing(this.latestInfo.attemptNumber())) { 0 until numPartitions } else { mapOutputTrackerMaster diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 2ff110e43bc7..06936e04dcec 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -137,9 +137,9 @@ private[scheduler] abstract class Stage( rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE } - def treatAllPartitionsMissing(attemptId: Int): Boolean = - this.attemptIdAllPartitionsMissing == attemptId + def areAllPartitionsMissing(attemptId: Int): Boolean = + this.attemptIdAllPartitionsMissing >= attemptId - def markAttemptIdForAllPartitionsMissing(attemptId: Int): Unit = - this.attemptIdAllPartitionsMissing = attemptId + def markAllPartitionsMissing(): Unit = + this.attemptIdAllPartitionsMissing = this.latestInfo.attemptNumber() } 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 afee74b16417..e4d63a1af098 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3277,11 +3277,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val failedStages = scheduler.failedStages.toSeq assert(failedStages.map(_.id) == Seq(1, 2)) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. - // TODO: THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all - // the partitions need to be retried - /* assert(failedStages.collect { - case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage - }.head.findMissingPartitions() == Seq(0)) */ + // As the ShuffleMapStage is inDeterminate all the partitions need to be retried assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0, 1)) @@ -4253,13 +4249,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val failedStages = scheduler.failedStages.toSeq assert(failedStages.map(_.id) == Seq(1, 2)) // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. - // TODO: THIS ASSERTION APPEARS TO BE WRONG. As the ShuffleMapStage is inDeterminate all - // the partitions need to be retried - /* - assert(failedStages.collect { - case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage - }.head.findMissingPartitions() == Seq(0)) - */ + // As the ShuffleMapStage is inDeterminate all the partitions need to be retried assert(failedStages.collect { case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage }.head.findMissingPartitions() == Seq(0, 1)) From 1ec98f0d1ba8eaea12fb4ba656783e03c6baca59 Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 11 Mar 2025 15:58:51 -0700 Subject: [PATCH 27/48] SPARK-51272. Fixed the situation where ResultStage is inDeterminate but because the ShuffleMap stage which has failed is deterministic, it misses the situation where possibly the other shuffle stage which is actually inDeterministic has also lost one of its partition --- .../apache/spark/scheduler/DAGScheduler.scala | 63 ++++++++++++------- 1 file changed, 41 insertions(+), 22 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 48cc239a86ac..421831ab685d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1873,15 +1873,29 @@ private[spark] class DAGScheduler( private[scheduler] def handleTaskCompletion(event: CompletionEvent): Unit = { val task = event.task val stageId = task.stageId + val stageOption = stageIdToStage.get(task.stageId) + val isIndeterministicZombie = event.reason match { + case Success if stageOption.isDefined => + val stage = stageOption.get + (task.stageAttemptId < stage.latestInfo.attemptNumber() + && stage.isIndeterminate) || stage.areAllPartitionsMissing (task.stageAttemptId) + + case _ => false + } + outputCommitCoordinator.taskCompleted( stageId, task.stageAttemptId, task.partitionId, event.taskInfo.attemptNumber, // this is a task attempt number - event.reason) + if (isIndeterministicZombie) { + TaskKilled(reason = "Indeterminate stage needs all tasks to be retried") + } else { + event.reason + }) - if (!stageIdToStage.contains(task.stageId)) { + if (stageOption.isEmpty) { // The stage may have already finished when we get this event -- e.g. maybe it was a // speculative task. It is important that we send the TaskEnd event in any case, so listeners // are properly notified and can chose to handle it. For instance, some listeners are @@ -1893,18 +1907,14 @@ private[spark] class DAGScheduler( return } - val stage = stageIdToStage(task.stageId) + val stage = stageOption.get // Make sure the task's accumulators are updated before any other processing happens, so that // we can post a task end event before any jobs or stages are updated. The accumulators are // only updated in certain cases. - val isIndeterministicZombie = event.reason match { + event.reason match { case Success => - val isZombieIndeterminate = - (task.stageAttemptId < stage.latestInfo.attemptNumber() - && stage.isIndeterminate) || - stage.areAllPartitionsMissing(task.stageAttemptId) - if (!isZombieIndeterminate) { + if (!isIndeterministicZombie) { task match { case rt: ResultTask[_, _] => val resultStage = stage.asInstanceOf[ResultStage] @@ -1919,18 +1929,15 @@ private[spark] class DAGScheduler( case _ => updateAccumulators(event) } } - isZombieIndeterminate - case _: ExceptionFailure | _: TaskKilled => - updateAccumulators(event) - false + case _: ExceptionFailure | _: TaskKilled => updateAccumulators(event) - case _ => false + case _ => } if (trackingCacheVisibility) { // Update rdd blocks' visibility status. blockManagerMaster.updateRDDBlockVisibility( - event.taskInfo.taskId, visible = event.reason == Success) + event.taskInfo.taskId, visible = event.reason == Success && !isIndeterministicZombie) } postTaskEnd(event) @@ -2133,6 +2140,12 @@ private[spark] class DAGScheduler( failedStages += failedStage failedStages += mapStage if (noResubmitEnqueued) { + 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 indeterminacy by " + + "checkpointing the RDD before repartition and try again." + } // 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 @@ -2159,13 +2172,6 @@ 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 indeterminacy by " + - "checkpointing the RDD before repartition and try again." - } - activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) // The stages will be rolled back after checking @@ -2203,6 +2209,19 @@ private[spark] class DAGScheduler( logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + log"we will roll back and rerun below stages which include itself and all its " + log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") + } else if (failedStage.isIndeterminate) { + failedStage match { + case resultStage: ResultStage if resultStage.activeJob.isDefined => + val numMissingPartitions = resultStage.findMissingPartitions().length + if (numMissingPartitions < resultStage.numTasks) { + // TODO: support to rollback result tasks. + abortStage(resultStage, generateErrorMessage(resultStage), None) + } else { + resultStage.markAllPartitionsMissing() + } + + case _ => + } } // We expect one executor failure to trigger many FetchFailures in rapid succession, From 69b66aa3be14caf40dc3089347f72fbea1163dc8 Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 11 Mar 2025 21:14:40 -0700 Subject: [PATCH 28/48] initial config --- .../scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index a3ae8c4f363a..0c97cc669b7c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -150,7 +150,7 @@ private object Spark51016Suite extends Logging { val outerjoin: DataFrame = getOuterJoinDF(spark) val correctRows = outerjoin.collect() JobListener.inKillMode = true - for (i <- 0 until 100) { + for (i <- 0 until 50) { try { eventually(timeout(3.minutes), interval(100.milliseconds)) { assert(sc.getExecutorIds().size == 2) @@ -180,7 +180,6 @@ private object Spark51016Suite extends Logging { }) assert(retriedResults.isEmpty) logInfo(s"found successful query exec on iter index = $i") - Thread.sleep(500) } catch { case se: SparkException if se.getMessage.contains("Please eliminate the" + " indeterminacy by checkpointing the RDD before repartition and try again") => From 3439bf947828c95a838a37490e4cad1a40aa9afd Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 11 Mar 2025 21:23:32 -0700 Subject: [PATCH 29/48] SPARK-51272. added disabled functional test to reproduce the issue. Needs fixes for SPARK-51272 and SPARK-51016 for it to consistently pass --- .../scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index 0c97cc669b7c..08d6d52a1e43 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -39,7 +39,7 @@ import org.apache.spark.{SparkContext, SparkException} @ExtendedYarnTest class Spark51016Suite extends BaseYarnClusterSuite { override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() - test("bug SPARK-51016 and SPARK-51272: Indeterminate stage retry giving wrong results") { + ignore("bug SPARK-51016 and SPARK-51272: Indeterminate stage retry giving wrong results") { testBasicYarnApp( Map( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", From abcd1331d5098c3b928cb176dda0e24367c1cd2b Mon Sep 17 00:00:00 2001 From: ashahid Date: Tue, 11 Mar 2025 23:18:53 -0700 Subject: [PATCH 30/48] SPARK-51272. added disabled functional test to reproduce the issue. Needs fixes for SPARK-51272 and SPARK-51016 for it to consistently pass --- .../org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala | 3 ++- .../scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala | 5 +++-- scalastyle-config.xml | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 59ec771316c0..ea3c07eef390 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -169,7 +169,8 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { extraEnv: Map[String, String] = Map(), outFile: Option[File] = None, testTimeOut: Int = 3, // minutes - timeOutIntervalCheck: Int = 1 /* seconds */): SparkAppHandle.State = { + timeOutIntervalCheck: Int = 1 // seconds + ): SparkAppHandle.State = { val deployMode = if (clientMode) "client" else "cluster" val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf) val env = Map( diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala index 08d6d52a1e43..68f1cca02cb6 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala @@ -26,15 +26,16 @@ import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.concurrent.Eventually._ +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageSubmitted} +import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType -import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} import org.apache.spark.tags.ExtendedYarnTest -import org.apache.spark.{SparkContext, SparkException} + @ExtendedYarnTest class Spark51016Suite extends BaseYarnClusterSuite { diff --git a/scalastyle-config.xml b/scalastyle-config.xml index ca3d507ed3a4..12593615a149 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -94,7 +94,7 @@ This file is divided into 3 sections: - + From b8865afaf89cae1a0dc23981286a01e5e9442df5 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 12 Mar 2025 06:55:45 -0700 Subject: [PATCH 31/48] SPARK-51272. added disabled functional test to reproduce the issue. Needs fixes for SPARK-51272 and SPARK-51016 for it to consistently pass --- .../yarn/{Spark51016Suite.scala => SparkHASuite.scala} | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) rename resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/{Spark51016Suite.scala => SparkHASuite.scala} (98%) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala similarity index 98% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala index 68f1cca02cb6..03c98fe19959 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/Spark51016Suite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.tags.ExtendedYarnTest @ExtendedYarnTest -class Spark51016Suite extends BaseYarnClusterSuite { +class SparkHASuite extends BaseYarnClusterSuite { override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() ignore("bug SPARK-51016 and SPARK-51272: Indeterminate stage retry giving wrong results") { testBasicYarnApp( @@ -64,7 +64,7 @@ class Spark51016Suite extends BaseYarnClusterSuite { val result = File.createTempFile("result", null, tempDir) val finalState = runSpark( clientMode = true, - mainClassName(Spark51016Suite.getClass), + mainClassName(SparkHASuite.getClass), appArgs = Seq(result.getAbsolutePath), extraConf = conf, testTimeOut = 30, @@ -73,7 +73,7 @@ class Spark51016Suite extends BaseYarnClusterSuite { } } -private object Spark51016Suite extends Logging { +private object SparkHASuite extends Logging { object Counter { var counter = 0 From 784aad3e22210195f3d4124fb72a9e4044355db6 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 12 Mar 2025 13:45:40 -0700 Subject: [PATCH 32/48] SPARK-51272. added another unit test which checks for all partitions of results to be retried, if result stage is inDeterminate but failed shuffle stage is determinate --- .../spark/scheduler/DAGSchedulerSuite.scala | 109 +++++++++++++++++- 1 file changed, 104 insertions(+), 5 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 e4d63a1af098..f8b7b36169a7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3165,14 +3165,17 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti null)) (shuffleId1, shuffleId2) } + private def constructTwoIndeterminateStage(): (Int, Int) = constructTwoStages(true, true) - private def constructTwoIndeterminateStage(): (Int, Int) = { - val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) + private def constructTwoStages( + stage1InDeterminate: Boolean, + stage2InDeterminate: Boolean): (Int, Int) = { + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = stage1InDeterminate) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) val shuffleId1 = shuffleDep1.shuffleId val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker, - indeterminate = true) + indeterminate = stage2InDeterminate) val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) val shuffleId2 = shuffleDep2.shuffleId @@ -3198,7 +3201,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } test("SPARK-51272: retry all the partitions of result stage, if the first result task" + - " has failed and ShuffleMap stage is inDeterminate") { + " has failed and failing ShuffleMap stage is inDeterminate") { val latch = new CountDownLatch(1) this.dagSchedulerInterceptor = new DagSchedulerInterceptor { override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { @@ -3233,6 +3236,8 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } val numPartitions = 2 + // The first shuffle stage is completed by the below function itself which creates two + // indeterminate stages. val (shuffleId1, shuffleId2) = constructTwoIndeterminateStage() completeShuffleMapStageSuccessfully(shuffleId2, 0, numPartitions) val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] @@ -3253,13 +3258,14 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti null)) val shuffleStage1 = this.scheduler.shuffleIdToMapStage(shuffleId1) val shuffleStage2 = this.scheduler.shuffleIdToMapStage(shuffleId2) - completeShuffleMapStageSuccessfully(0, 1, numPartitions) import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ eventually(timeout(3.minutes), interval(500.milliseconds)) { shuffleStage1.latestInfo.attemptNumber() should equal(1) } + completeShuffleMapStageSuccessfully(0, 1, numPartitions) + eventually(timeout(3.minutes), interval(500.milliseconds)) { shuffleStage2.latestInfo.attemptNumber() should equal(1) } @@ -3268,6 +3274,99 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti resultStage.latestInfo.attemptNumber() should equal(1) } org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == 2) + org.scalatest.Assertions.assert(resultStage.findMissingPartitions().size == 2) + } + + test("SPARK-51272: retry all the partitions of result stage, if the first result task" + + " has failed with failing ShuffleStage determinate but result stage has another ShuffleStage" + + " which is indeterminate") { + val latch = new CountDownLatch(1) + this.dagSchedulerInterceptor = new DagSchedulerInterceptor { + override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { + event match { + case ResubmitFailedStages => + // Before the ResubmitFailedStages is added to the queue, add the successful + // partition task completion. + runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) + latch.countDown() + + case _ => + } + } + + override def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = { + event match { + case CompletionEvent(_, reason, _, _, _, _) => + reason match { + case FetchFailed(_, _, _, _, _, _) => + // Do not allow this thread to exit, till the ResubmitFailedStages + // in callback is received. This is to ensure that this thread + // does not exit and process the ResubmitFailedStage event, before + // the queue gets successful partition task completion + latch.await(50, TimeUnit.SECONDS) + + case _ => + } + + case _ => + } + } + } + + val numPartitions = 2 + // The first shuffle stage is completed by the below function itself which creates two + // stages. + val (detShuffleId1, indetShuffleId2) = constructTwoStages(stage1InDeterminate = false, + stage2InDeterminate = true) + completeShuffleMapStageSuccessfully(indetShuffleId2, 0, numPartitions) + assert(mapOutputTracker.findMissingPartitions(indetShuffleId2) === Some(Seq.empty)) + val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] + val activeJob = resultStage.activeJob + assert(activeJob.isDefined) + // The result stage is still waiting for its 2 tasks to complete + assert(resultStage.findMissingPartitions() == Seq.tabulate(numPartitions)(i => i)) + + // The below event will cause the first task of result stage to fail. + // Below scenario should happen if behaving correctly: + // Since the result stage is dependent on two shuffles of which 1 is inDeterminate, + // the retry of the ResultStage should be for both tasks, even if the failed shuffle stage + // is deterministic, as there is no guarantee at this point, if the indeterminate shuffle + // stage 2 has also failed or not. If inDeterminate stage too has hypothetically failed for, + // for first result partition1, but successful for result partition2, then re-execution of + // of shuffle stage 2 ( indeterminate) , will cause wrong results. So to avoid this, once + // an inDeterminate Result Stage is being retried, no successful partitions should be + // accepted having stale attempt + // + runEvent( + makeCompletionEvent( + taskSets(2).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), detShuffleId1, 0L, 0, 0, "ignored"), + null)) + val detShuffleStage1 = this.scheduler.shuffleIdToMapStage(detShuffleId1) + val inDetshuffleStage2 = this.scheduler.shuffleIdToMapStage(indetShuffleId2) + import org.scalatest.concurrent.Eventually._ + import org.scalatest.matchers.should.Matchers._ + import org.scalatest.time.SpanSugar._ + eventually(timeout(3.minutes), interval(500.milliseconds)) { + detShuffleStage1.latestInfo.attemptNumber() should equal(1) + } + completeShuffleMapStageSuccessfully(0, 1, numPartitions) + + // Though the inDetShuffleStage2 has not suffered any loss, but source code of DagScheduler + // has code to remove shuffleoutputs based on the lost BlockManager , which in this case will + // result in loss of output of shuffle2 also. It looses one partition and hence will be + // re-attempted.. + // But that re-attempt should fetch all partitions! + eventually(timeout(3.minutes), interval(500.milliseconds)) { + inDetshuffleStage2.latestInfo.attemptNumber() should equal(1) + } + org.scalatest.Assertions.assert(inDetshuffleStage2.latestInfo.numTasks == 2) + org.scalatest.Assertions.assert(inDetshuffleStage2.findMissingPartitions().size == 2) + completeShuffleMapStageSuccessfully(1, 1, numPartitions) + eventually(timeout(3.minutes), interval(500.milliseconds)) { + resultStage.latestInfo.attemptNumber() should equal(1) + } + org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == 2) } test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { From cd041ee1dae2328fd313b855869ae97d795be8ac Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 12 Mar 2025 18:26:09 -0700 Subject: [PATCH 33/48] SPARK-51272. formatting change --- .../src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 1 - 1 file changed, 1 deletion(-) 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 421831ab685d..c1bc4c5b2594 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1883,7 +1883,6 @@ private[spark] class DAGScheduler( case _ => false } - outputCommitCoordinator.taskCompleted( stageId, task.stageAttemptId, From d77bcd8bbcc43375afd578c8d8fafbb8dfd8d8d3 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 12 Mar 2025 18:32:13 -0700 Subject: [PATCH 34/48] SPARK-51272. SPARK-51016. combined PRs with HA Test enabled --- .../test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala index 03c98fe19959..5e41d96c299a 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala @@ -40,7 +40,7 @@ import org.apache.spark.tags.ExtendedYarnTest @ExtendedYarnTest class SparkHASuite extends BaseYarnClusterSuite { override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() - ignore("bug SPARK-51016 and SPARK-51272: Indeterminate stage retry giving wrong results") { + test("bug SPARK-51016 and SPARK-51272: Indeterminate stage retry giving wrong results") { testBasicYarnApp( Map( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", From 2c9527146e9e1e9e8d3d95c90e2d9a22e0d4aada Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 13 Mar 2025 11:15:56 -0700 Subject: [PATCH 35/48] SPARK-51272. Increased the number of iterations. Using two positions ( stage submitted and stage completed to kill the executors) --- .../spark/deploy/yarn/SparkHASuite.scala | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala index 03c98fe19959..16e399385a69 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala @@ -19,17 +19,14 @@ package org.apache.spark.deploy.yarn import java.io.File import java.nio.charset.StandardCharsets - import scala.concurrent.duration._ - import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.concurrent.Eventually._ - import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageSubmitted} +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted} import org.apache.spark.sql.{DataFrame, Encoders, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -54,8 +51,7 @@ class SparkHASuite extends BaseYarnClusterSuite { EXECUTOR_CORES.key -> "1", EXECUTOR_MEMORY.key -> "512m", EXECUTOR_INSTANCES.key -> "2", - "spark.ui.port" -> "4040", - "spark.ui.enabled" -> "true", + "spark.ui.enabled" -> "false", "spark.yarn.max.executor.failures" -> "100000" )) } @@ -151,7 +147,11 @@ private object SparkHASuite extends Logging { val outerjoin: DataFrame = getOuterJoinDF(spark) val correctRows = outerjoin.collect() JobListener.inKillMode = true - for (i <- 0 until 50) { + JobListener.killWhen = KillPosition.KILL_IN_STAGE_SUBMISSION + for (i <- 0 until 100) { + if (i > 49) { + JobListener.killWhen = KillPosition.KILL_IN_STAGE_COMPLETION + } try { eventually(timeout(3.minutes), interval(100.milliseconds)) { assert(sc.getExecutorIds().size == 2) @@ -238,7 +238,18 @@ private[spark] class JobListener extends SparkListener with Logging { } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { - if (stageSubmitted.stageInfo.shuffleDepId.isEmpty && pidToKill.nonEmpty) { + if (stageSubmitted.stageInfo.shuffleDepId.isEmpty && pidToKill.nonEmpty && + JobListener.killWhen == KillPosition.KILL_IN_STAGE_SUBMISSION) { + val pid = pidToKill.get + pidToKill = None + logInfo(s"killing executor for pid = $pid") + PIDGetter.killExecutor(pid) + } + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + if (stageCompleted.stageInfo.shuffleDepId.exists(_ % 2 == count % 2) && pidToKill.nonEmpty && + JobListener.killWhen == KillPosition.KILL_IN_STAGE_COMPLETION) { val pid = pidToKill.get pidToKill = None logInfo(s"killing executor for pid = $pid") @@ -247,7 +258,16 @@ private[spark] class JobListener extends SparkListener with Logging { } } +object KillPosition extends Enumeration { + type KillPosition = Value + val KILL_IN_STAGE_SUBMISSION, KILL_IN_STAGE_COMPLETION, NONE = Value +} + object JobListener { @volatile var inKillMode: Boolean = false + + import KillPosition._ + @volatile + var killWhen: KillPosition = NONE } From 1014c6babf57d08ba66bba3706bd350f4e9277dd Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 13 Mar 2025 12:59:11 -0700 Subject: [PATCH 36/48] SPARK-51272. fixed scalastyle issue --- .../test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala index 16e399385a69..8b217d8f2455 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala @@ -19,10 +19,13 @@ package org.apache.spark.deploy.yarn import java.io.File import java.nio.charset.StandardCharsets + import scala.concurrent.duration._ + import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.concurrent.Eventually._ + import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_INSTANCES, EXECUTOR_MEMORY} From b1a0593e576ecaa671e664304d2bf17302411b0e Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 13 Mar 2025 18:41:47 -0700 Subject: [PATCH 37/48] SPARK-51272. Implementing review feedbackk. Making sql dependency in yarn with test scope --- resource-managers/yarn/pom.xml | 1 + .../spark/deploy/yarn/BaseYarnClusterSuite.scala | 12 +++++++++--- .../org/apache/spark/deploy/yarn/SparkHASuite.scala | 3 +-- scalastyle-config.xml | 2 +- 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index d4bed6103491..5b825dbe103e 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -41,6 +41,7 @@ org.apache.spark spark-sql_${scala.binary.version} ${project.version} + test org.apache.spark diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index ea3c07eef390..84f37f2524e1 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -168,8 +168,7 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { extraConf: Map[String, String] = Map(), extraEnv: Map[String, String] = Map(), outFile: Option[File] = None, - testTimeOut: Int = 3, // minutes - timeOutIntervalCheck: Int = 1 // seconds + testTimeOutParams: TimeoutParams = TimeoutParams.DEFAULT ): SparkAppHandle.State = { val deployMode = if (clientMode) "client" else "cluster" val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf) @@ -215,7 +214,8 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { val handle = launcher.startApplication() try { - eventually(timeout(testTimeOut.minutes), interval(timeOutIntervalCheck.second)) { + eventually(timeout(testTimeOutParams.testTimeOut), + interval(testTimeOutParams.timeOutIntervalCheck)) { assert(handle.getState().isFinal()) } } finally { @@ -300,3 +300,9 @@ abstract class BaseYarnClusterSuite extends SparkFunSuite with Matchers { } } + +case class TimeoutParams(testTimeOut: Duration, timeOutIntervalCheck: Duration) + +object TimeoutParams { + val DEFAULT = TimeoutParams(3.minutes, 1.seconds) +} \ No newline at end of file diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala index 8b217d8f2455..ca23736098be 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/SparkHASuite.scala @@ -66,8 +66,7 @@ class SparkHASuite extends BaseYarnClusterSuite { mainClassName(SparkHASuite.getClass), appArgs = Seq(result.getAbsolutePath), extraConf = conf, - testTimeOut = 30, - timeOutIntervalCheck = 30) + testTimeOutParams = TimeoutParams(30.minutes, 30.seconds)) checkResult(finalState, result) } } diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 12593615a149..ca3d507ed3a4 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -94,7 +94,7 @@ This file is divided into 3 sections: - + From dc08a7b7f1da9b93cd1c005576dfc29f4b464b48 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 13 Mar 2025 18:43:07 -0700 Subject: [PATCH 38/48] SPARK-51272. Implementing review feedbackk. Making sql dependency in yarn with test scope --- .../org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 84f37f2524e1..7d1f4d7a989a 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -305,4 +305,4 @@ case class TimeoutParams(testTimeOut: Duration, timeOutIntervalCheck: Duration) object TimeoutParams { val DEFAULT = TimeoutParams(3.minutes, 1.seconds) -} \ No newline at end of file +} From 4f263d1b9ed8ad79df1f8cea1ad95fef3a60779c Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 14 Mar 2025 00:07:33 -0700 Subject: [PATCH 39/48] SPARK-51272. refactored the code so that findMissingPartitions code remains unchanged --- .../org/apache/spark/scheduler/ResultStage.scala | 12 +++++++----- .../apache/spark/scheduler/ShuffleMapStage.scala | 13 +++++++------ .../scala/org/apache/spark/scheduler/Stage.scala | 6 +++++- 3 files changed, 19 insertions(+), 12 deletions(-) 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 a364f5ad21e4..8463f4ae30e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -61,11 +61,13 @@ private[spark] class ResultStage( */ override def findMissingPartitions(): Seq[Int] = { val job = activeJob.get - val allPartitions = (0 until job.numPartitions) - if (this.areAllPartitionsMissing(this.latestInfo.attemptNumber())) { - allPartitions - } else { - allPartitions.filter(id => !job.finished(id)) + (0 until job.numPartitions).filter(id => !job.finished(id)) + } + + override protected def basicMarkAllPartitionsMissing(): Unit = { + val job = activeJob.get + for(id <- 0 until job.numPartitions) { + job.finished(id) = false } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 5bbbf34d4453..9af975fc9105 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -90,11 +90,12 @@ private[spark] class ShuffleMapStage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ override def findMissingPartitions(): Seq[Int] = { - if (this.areAllPartitionsMissing(this.latestInfo.attemptNumber())) { - 0 until numPartitions - } else { - mapOutputTrackerMaster - .findMissingPartitions(shuffleDep.shuffleId).getOrElse(0 until numPartitions) - } + mapOutputTrackerMaster + .findMissingPartitions(shuffleDep.shuffleId) + .getOrElse(0 until numPartitions) + } + + override protected def basicMarkAllPartitionsMissing(): Unit = { + mapOutputTrackerMaster.unregisterAllMapAndMergeOutput(this.shuffleDep.shuffleId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 06936e04dcec..cac1c5bed013 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -140,6 +140,10 @@ private[scheduler] abstract class Stage( def areAllPartitionsMissing(attemptId: Int): Boolean = this.attemptIdAllPartitionsMissing >= attemptId - def markAllPartitionsMissing(): Unit = + def markAllPartitionsMissing(): Unit = { this.attemptIdAllPartitionsMissing = this.latestInfo.attemptNumber() + this.basicMarkAllPartitionsMissing() + } + + protected def basicMarkAllPartitionsMissing(): Unit } From 3895fec057bd2a205a3a59133698546e7b165962 Mon Sep 17 00:00:00 2001 From: ashahid Date: Fri, 14 Mar 2025 19:06:21 -0700 Subject: [PATCH 40/48] SPARK-51272. refactored the code, based on the review feedback. Refactored the tests --- .../apache/spark/scheduler/DAGScheduler.scala | 8 +- .../apache/spark/scheduler/ResultStage.scala | 19 +- .../spark/scheduler/ShuffleMapStage.scala | 4 - .../org/apache/spark/scheduler/Stage.scala | 15 +- .../spark/scheduler/DAGSchedulerSuite.scala | 164 ++++++++++-------- 5 files changed, 119 insertions(+), 91 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 c1bc4c5b2594..4b63f181a1aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1878,7 +1878,7 @@ private[spark] class DAGScheduler( case Success if stageOption.isDefined => val stage = stageOption.get (task.stageAttemptId < stage.latestInfo.attemptNumber() - && stage.isIndeterminate) || stage.areAllPartitionsMissing (task.stageAttemptId) + && stage.isIndeterminate) || stage.shouldDiscardResult(task.stageAttemptId) case _ => false } @@ -2188,10 +2188,12 @@ private[spark] class DAGScheduler( abortStage(mapStage, reason, None) } else { rollingBackStages += mapStage - mapStage.markAllPartitionsMissing() + mapOutputTracker.unregisterAllMapAndMergeOutput( + mapStage.shuffleDep.shuffleId) } } else { - mapStage.markAllPartitionsMissing() + mapOutputTracker.unregisterAllMapAndMergeOutput( + mapStage.shuffleDep.shuffleId) } case resultStage: ResultStage if resultStage.activeJob.isDefined => 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 8463f4ae30e8..92f76f9adec8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -38,6 +38,9 @@ private[spark] class ResultStage( resourceProfileId: Int) extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite, resourceProfileId) { + @volatile + private var discardResultsForAttemptId: Int = -1 + /** * The active job for this result stage. Will be empty if the job has already finished * (e.g., because the job was cancelled). @@ -54,6 +57,14 @@ private[spark] class ResultStage( _activeJob = None } + override def makeNewStageAttempt( + numPartitionsToCompute: Int, + taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = { + super.makeNewStageAttempt(numPartitionsToCompute, taskLocalityPreferences) + // clear the attemptId set in the attemptIdAllPartitionsMissing + discardResultsForAttemptId = -1 + } + /** * Returns the sequence of partition ids that are missing (i.e. needs to be computed). * @@ -64,12 +75,16 @@ private[spark] class ResultStage( (0 until job.numPartitions).filter(id => !job.finished(id)) } - override protected def basicMarkAllPartitionsMissing(): Unit = { + def markAllPartitionsMissing(): Unit = { + this.discardResultsForAttemptId = this.latestInfo.attemptNumber() val job = activeJob.get - for(id <- 0 until job.numPartitions) { + for (id <- 0 until job.numPartitions) { job.finished(id) = false } } + override def shouldDiscardResult(attemptId: Int): Boolean = + this.discardResultsForAttemptId >= attemptId + override def toString: String = "ResultStage " + id } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 9af975fc9105..db09d19d0acf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -94,8 +94,4 @@ private[spark] class ShuffleMapStage( .findMissingPartitions(shuffleDep.shuffleId) .getOrElse(0 until numPartitions) } - - override protected def basicMarkAllPartitionsMissing(): Unit = { - mapOutputTrackerMaster.unregisterAllMapAndMergeOutput(this.shuffleDep.shuffleId) - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index cac1c5bed013..f8420f45482f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -63,9 +63,6 @@ private[scheduler] abstract class Stage( val resourceProfileId: Int) extends Logging { - @volatile - private var attemptIdAllPartitionsMissing: Int = -1 - val numPartitions = rdd.partitions.length /** Set of jobs that this stage belongs to. */ @@ -109,8 +106,6 @@ private[scheduler] abstract class Stage( this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, resourceProfileId = resourceProfileId) nextAttemptId += 1 - // clear the attemptId set in the attemptIdAllPartitionsMissing - attemptIdAllPartitionsMissing = -1 } /** Forward the nextAttemptId if skipped and get visited for the first time. */ @@ -137,13 +132,5 @@ private[scheduler] abstract class Stage( rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE } - def areAllPartitionsMissing(attemptId: Int): Boolean = - this.attemptIdAllPartitionsMissing >= attemptId - - def markAllPartitionsMissing(): Unit = { - this.attemptIdAllPartitionsMissing = this.latestInfo.attemptNumber() - this.basicMarkAllPartitionsMissing() - } - - protected def basicMarkAllPartitionsMissing(): Unit + def shouldDiscardResult(attemptId: Int): Boolean = false } 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 f8b7b36169a7..5d4f759d83bb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -68,6 +68,7 @@ class DAGSchedulerEventProcessLoopTester( // buffer events for sequent processing later instead of processing them recursively. dagSchedulerInterceptorOpt.foreach(_.beforeAddingDagEventToQueue(event)) eventQueue += event + dagSchedulerInterceptorOpt.foreach(_.afterAddingDagEventToQueue(event)) } else { try { isProcessing = true @@ -181,6 +182,7 @@ class DAGSchedulerSuiteDummyException extends Exception trait DagSchedulerInterceptor { def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = {} + def afterAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = {} def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = {} } @@ -3200,45 +3202,18 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti "Spark can only do this while using the new shuffle block fetching protocol")) } - test("SPARK-51272: retry all the partitions of result stage, if the first result task" + - " has failed and failing ShuffleMap stage is inDeterminate") { - val latch = new CountDownLatch(1) - this.dagSchedulerInterceptor = new DagSchedulerInterceptor { - override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { - event match { - case ResubmitFailedStages => - // Before the ResubmitFailedStages is added to the queue, add the successful - // partition task completion. - runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) - latch.countDown() - - case _ => - } - } - - override def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = { - event match { - case CompletionEvent(_, reason, _, _, _, _) => - reason match { - case FetchFailed(_, _, _, _, _, _) => - // Do not allow this thread to exit, till the ResubmitFailedStages - // in callback is received. This is to ensure that this thread - // does not exit and process the ResubmitFailedStage event, before - // the queue gets successful partition task completion - latch.await(50, TimeUnit.SECONDS) - case _ => - } - case _ => - } - } - } + test("SPARK-51272: retry all the partitions of result stage, if the first result task" + + " has failed and failing ShuffleMap stage is inDeterminate") { + this.dagSchedulerInterceptor = createDagInterceptorForSpark51272( + () => taskSets(2).tasks(1), "RELEASE_LATCH") val numPartitions = 2 // The first shuffle stage is completed by the below function itself which creates two // indeterminate stages. - val (shuffleId1, shuffleId2) = constructTwoIndeterminateStage() + val (shuffleId1, shuffleId2) = constructTwoStages( + stage1InDeterminate = false, stage2InDeterminate = true) completeShuffleMapStageSuccessfully(shuffleId2, 0, numPartitions) val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] val activeJob = resultStage.activeJob @@ -3273,45 +3248,15 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti eventually(timeout(3.minutes), interval(500.milliseconds)) { resultStage.latestInfo.attemptNumber() should equal(1) } - org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == 2) - org.scalatest.Assertions.assert(resultStage.findMissingPartitions().size == 2) + org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == numPartitions) + org.scalatest.Assertions.assert(resultStage.findMissingPartitions().size == numPartitions) } test("SPARK-51272: retry all the partitions of result stage, if the first result task" + " has failed with failing ShuffleStage determinate but result stage has another ShuffleStage" + " which is indeterminate") { - val latch = new CountDownLatch(1) - this.dagSchedulerInterceptor = new DagSchedulerInterceptor { - override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { - event match { - case ResubmitFailedStages => - // Before the ResubmitFailedStages is added to the queue, add the successful - // partition task completion. - runEvent(makeCompletionEvent(taskSets(2).tasks(1), Success, 11)) - latch.countDown() - - case _ => - } - } - - override def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = { - event match { - case CompletionEvent(_, reason, _, _, _, _) => - reason match { - case FetchFailed(_, _, _, _, _, _) => - // Do not allow this thread to exit, till the ResubmitFailedStages - // in callback is received. This is to ensure that this thread - // does not exit and process the ResubmitFailedStage event, before - // the queue gets successful partition task completion - latch.await(50, TimeUnit.SECONDS) - - case _ => - } - - case _ => - } - } - } + this.dagSchedulerInterceptor = createDagInterceptorForSpark51272( + () => taskSets(2).tasks(1), "RELEASE_LATCH") val numPartitions = 2 // The first shuffle stage is completed by the below function itself which creates two @@ -3326,6 +3271,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // The result stage is still waiting for its 2 tasks to complete assert(resultStage.findMissingPartitions() == Seq.tabulate(numPartitions)(i => i)) + // The below event will cause the first task of result stage to fail. // Below scenario should happen if behaving correctly: // Since the result stage is dependent on two shuffles of which 1 is inDeterminate, @@ -3366,7 +3312,41 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti eventually(timeout(3.minutes), interval(500.milliseconds)) { resultStage.latestInfo.attemptNumber() should equal(1) } - org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == 2) + org.scalatest.Assertions.assert(resultStage.latestInfo.numTasks == numPartitions) + } + + test("SPARK-51272: retry all the partitions of Shuffle stage, if any task of ShuffleStage " + + " has failed and failing ShuffleMap stage is inDeterminate") { + val numPartitions = 2 + this.dagSchedulerInterceptor = createDagInterceptorForSpark51272( + () => taskSets(1).tasks(1), makeMapStatus(host = "hostZZZ", reduces = numPartitions)) + // The first shuffle stage is completed by the below function itself which creates two + // indeterminate stages. + val (shuffleId1, shuffleId2) = constructTwoStages( + stage1InDeterminate = false, stage2InDeterminate = true) + // This will trigger the resubmit failed stage and in before adding resubmit message to the + // queue, a successful partition completion event will arrive. + + runEvent( + makeCompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId2, 0L, 0, 0, "ignored"), + null)) + + // run completion task for first stage as the Fetch Failed for second shuffle stage is sharing + // the block Id + completeShuffleMapStageSuccessfully(0, 1, numPartitions) + + val shuffleStage2 = scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage] + + import org.scalatest.concurrent.Eventually._ + import org.scalatest.matchers.should.Matchers._ + import org.scalatest.time.SpanSugar._ + + eventually(timeout(3.minutes), interval(500.milliseconds)) { + shuffleStage2.latestInfo.attemptNumber() should equal(1) + } + org.scalatest.Assertions.assert(shuffleStage2.findMissingPartitions().size == numPartitions) } test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { @@ -5321,6 +5301,54 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } CompletionEvent(task, reason, result, accumUpdates ++ extraAccumUpdates, metricPeaks, taskInfo) } + + private def createDagInterceptorForSpark51272(latchReleaseTask: () => Task[_], taskResult: Any): + DagSchedulerInterceptor = { + new DagSchedulerInterceptor { + val latch = new CountDownLatch(1) + override def beforeAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { + event match { + case ResubmitFailedStages => + // Before the ResubmitFailedStages is added to the queue, add the successful + // partition task completion. + runEvent(makeCompletionEvent(latchReleaseTask(), Success, taskResult)) + + case _ => + } + } + + override def afterAddingDagEventToQueue(event: DAGSchedulerEvent): Unit = { + event match { + case CompletionEvent(_, reason, result, _, _, _) => + reason match { + case Success if result == taskResult => latch.countDown() + + case _ => + } + + case _ => + } + } + + override def afterDirectProcessingOfDagEvent(event: DAGSchedulerEvent): Unit = { + event match { + case CompletionEvent(_, reason, _, _, _, _) => + reason match { + case FetchFailed(_, _, _, _, _, _) => + // Do not allow this thread to exit, till spurious sucessfull task + // ( latchRelease task gets in the queue). This would ensure that + // ResubmitFailedStages task will always be processed after the spurious task + // is processed. + latch.await(50, TimeUnit.SECONDS) + + case _ => + } + + case _ => + } + } + } + } } class DAGSchedulerAbortStageOffSuite extends DAGSchedulerSuite { From 2498169c812f8eba0c79ed8dd3cdaaa067f3a891 Mon Sep 17 00:00:00 2001 From: ashahid Date: Sat, 15 Mar 2025 00:11:02 -0700 Subject: [PATCH 41/48] SPARK-51272. Aborting the result stage if its number of missing partitions is not equal to total partitions and resultstage is inDeterminate. Though with the current PR, this is not expected, unless there still remains a window of race. --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 6 ++++++ 1 file changed, 6 insertions(+) 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 4b63f181a1aa..454d8cd693d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1554,6 +1554,12 @@ private[spark] class DAGScheduler( case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) sms.shuffleDep.newShuffleMergeState() + + case rs: ResultStage if rs.isIndeterminate + && rs.numPartitions != rs.findMissingPartitions().size => + abortStage(rs, "There exists a race condition, which has resulted in a result task" + + " getting committed, which should have been discarded", None) + case _ => } From 9fd7114dde30ea386f783675e3bcb98bf730bbe1 Mon Sep 17 00:00:00 2001 From: ashahid Date: Mon, 17 Mar 2025 12:39:36 -0700 Subject: [PATCH 42/48] SPARK-51272. reverted previous change of throwing abort exception --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 5 ----- 1 file changed, 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 454d8cd693d0..0693b9a00a35 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1555,11 +1555,6 @@ private[spark] class DAGScheduler( mapOutputTracker.unregisterAllMapAndMergeOutput(sms.shuffleDep.shuffleId) sms.shuffleDep.newShuffleMergeState() - case rs: ResultStage if rs.isIndeterminate - && rs.numPartitions != rs.findMissingPartitions().size => - abortStage(rs, "There exists a race condition, which has resulted in a result task" + - " getting committed, which should have been discarded", None) - case _ => } From 54f18fe1a380ba6409d97f444d9f0fb21453d871 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 19 Mar 2025 11:09:48 -0700 Subject: [PATCH 43/48] SPARK-51016. Implemented review feedback. Thank you @squito --- .../spark/sql/catalyst/expressions/Expression.scala | 11 +++++++++++ .../catalyst/expressions/ExpressionEvalHelper.scala | 2 +- .../catalyst/expressions/NondeterministicSuite.scala | 8 ++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 4922713de408..3a5beb2362af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -115,6 +115,17 @@ abstract class Expression extends TreeNode[Expression] { */ lazy val deterministic: Boolean = children.forall(_.deterministic) + /** + * The information conveyed by this method hasIndeterminism differs from that conveyed + * by [[deterministic]] in the way that an [[Attribute]] representing an [[Expression]] having + * [[deterministic]] flag false, would have its [[deterministic]] flag true, but it would still + * have [[hasIndeterminism]] as true. Because the Attribute's evaluation represents a quantity + * which constitutes inDeterminism. Contrasted with [[deterministic]] flag which is always true + * for Leaf Expressions like [[AttributeReference]], [[hasIndeterminism]] carries information + * about the nature of the evaluated value, represented by the [[Expression]] + * @return Boolean true if the expression's evaluated value is a result of some indeterministic + * quantity. + */ def hasIndeterminism: Boolean = _hasIndeterminism @transient diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 2347410d4537..d8da04ef6456 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -76,7 +76,7 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB case _ => expr.mapChildren(replace) } - def prepareEvaluation(expression: Expression): Expression = { + private[expressions] def prepareEvaluation(expression: Expression): Expression = { val serializer = new JavaSerializer(new SparkConf()).newInstance() val resolver = ResolveTimeZone val expr = replace(resolver.resolveTimeZones(expression)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala index 2f0aa5fa6875..29f600347ea0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NondeterministicSuite.scala @@ -40,7 +40,13 @@ class NondeterministicSuite extends SparkFunSuite with ExpressionEvalHelper { assertIndeterminancyComponent(MonotonicallyIncreasingID()) val alias = Alias(Multiply(MonotonicallyIncreasingID(), Literal(100L)), "al1")() assertIndeterminancyComponent(alias) + // For the attribute created from an Alias with deterministic flag false, the attribute would + // carry forward that information from Alias, via the hasIndeterminism flag value being true. assertIndeterminancyComponent(alias.toAttribute) + // But the Attribute's deterministic flag would be true ( implying it does not carry forward + // that inDeterministic nature of evaluated quantity which Attribute represents) + assert(prepareEvaluation(alias.toAttribute).deterministic) + assertIndeterminancyComponent(Multiply(alias.toAttribute, Literal(1000L))) assertIndeterminancyComponent( HashPartitioning(Seq(Multiply(MonotonicallyIncreasingID(), Literal(100L))), 5)) @@ -65,4 +71,6 @@ class NondeterministicSuite extends SparkFunSuite with ExpressionEvalHelper { RangePartitioning(Seq(SortOrder.apply(alias.toAttribute, Descending)), 5)) assertNoIndeterminancyComponent(KeyGroupedPartitioning(Seq(alias.toAttribute), 5)) } + + } From 56e572a7915f64a21e9e1cb0a8e7e710571522fc Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 20 Mar 2025 15:32:50 -0700 Subject: [PATCH 44/48] SPARK-51016. Implemented review feedback from @squito and @mridulm --- .../scala/org/apache/spark/Dependency.scala | 22 ++----- .../main/scala/org/apache/spark/rdd/RDD.scala | 3 - .../spark/scheduler/ShuffleMapStage.scala | 2 - .../exchange/ShuffleExchangeExec.scala | 66 ++++++++++++++----- 4 files changed, 53 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 2751d7b3b2e0..573608c4327e 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -79,25 +79,13 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( @transient private val _rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, - val serializer: Serializer, - val keyOrdering: Option[Ordering[K]], - val aggregator: Option[Aggregator[K, V, C]], - val mapSideCombine: Boolean, - val shuffleWriterProcessor: ShuffleWriteProcessor, - val isInDeterministic: Boolean) + 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 shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor) extends Dependency[Product2[K, V]] with Logging { - def this ( - rdd: RDD[_ <: Product2[K, V]], - partitioner: Partitioner, - serializer: Serializer = SparkEnv.get.serializer, - keyOrdering: Option[Ordering[K]] = None, - aggregator: Option[Aggregator[K, V, C]] = None, - mapSideCombine: Boolean = false, - shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor - ) = this(rdd, partitioner, serializer, keyOrdering, aggregator, mapSideCombine, - shuffleWriterProcessor, false) - if (mapSideCombine) { require(aggregator.isDefined, "Map-side combine without Aggregator specified!") } 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 2ccc0f6476af..80db818b77e4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -2103,9 +2103,6 @@ abstract class RDD[T: ClassTag]( @DeveloperApi protected def getOutputDeterministicLevel: DeterministicLevel.Value = { val deterministicLevelCandidates = dependencies.map { - case dep: ShuffleDependency[_, _, _] if dep.isInDeterministic => - DeterministicLevel.INDETERMINATE - // The shuffle is not really happening, treat it like narrow dependency and assume the output // deterministic level of current RDD is same as parent. case dep: ShuffleDependency[_, _, _] if dep.rdd.partitioner.exists(_ == dep.partitioner) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 38da37751206..db09d19d0acf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -94,6 +94,4 @@ private[spark] class ShuffleMapStage( .findMissingPartitions(shuffleDep.shuffleId) .getOrElse(0 until numPartitions) } - - override def isIndeterminate: Boolean = this.shuffleDep.isInDeterministic || super.isIndeterminate } 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 bf63d2772c87..e5a3e935c691 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 @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution.exchange import java.util.concurrent.atomic.AtomicReference import java.util.function.Supplier - import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future, Promise} - import org.apache.spark._ import org.apache.spark.internal.config -import org.apache.spark.rdd.{RDD, RDDOperationScope} +import org.apache.spark.rdd.{DeterministicLevel, MapPartitionsRDD, RDD, RDDOperationScope} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProcessor} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -43,6 +41,8 @@ import org.apache.spark.util.{MutablePair, ThreadUtils} import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} import org.apache.spark.util.random.XORShiftRandom +import scala.reflect.ClassTag + /** * Common trait for all shuffle exchange implementations to facilitate pattern matching. */ @@ -452,25 +452,37 @@ object ShuffleExchangeExec { rdd } + val isIndeterministic = newPartitioning match { + case expr: Expression => expr.hasIndeterminism + case _ => false + } + // round-robin function is order sensitive if we don't sort the input. val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition if (needToCopyObjectsBeforeShuffle(part)) { - newRdd.mapPartitionsWithIndexInternal((_, iter) => { - val getPartitionKey = getPartitionKeyExtractor() - iter.map { row => (part.getPartition(getPartitionKey(row)), row.copy()) } - }, isOrderSensitive = isOrderSensitive) + this.createRddWithPartition( + newRdd, + (_, iter: Iterator[InternalRow]) => { + val getPartitionKey = getPartitionKeyExtractor() + iter.map { row => (part.getPartition(getPartitionKey(row)), row.copy()) } + }, + isIndeterministic, + isOrderSensitive + ) + } else { - newRdd.mapPartitionsWithIndexInternal((_, iter) => { - val getPartitionKey = getPartitionKeyExtractor() - val mutablePair = new MutablePair[Int, InternalRow]() - iter.map { row => mutablePair.update(part.getPartition(getPartitionKey(row)), row) } - }, isOrderSensitive = isOrderSensitive) + this.createRddWithPartition( + newRdd, + (_, iter: Iterator[InternalRow]) => { + val getPartitionKey = getPartitionKeyExtractor() + val mutablePair = new MutablePair[Int, InternalRow]() + iter.map { row => mutablePair.update(part.getPartition(getPartitionKey(row)), row) } + }, + isIndeterministic, + isOrderSensitive) } } - val isIndeterministic = newPartitioning match { - case expr: Expression => expr.hasIndeterminism - case _ => false - } + // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds // are in the form of (partitionId, row) and every partitionId is in the expected range // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough. @@ -482,12 +494,30 @@ object ShuffleExchangeExec { None, None, false, - shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics), - isIndeterministic) + shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics)) dependency } + private def createRddWithPartition[T: ClassTag, U: ClassTag]( + rdd: RDD[T], + f: (Int, Iterator[T]) => Iterator[U], + isInDeterminate: Boolean, + isOrderSensitive: Boolean): RDD[U] = if (isInDeterminate) { + RDDOperationScope.withScope(rdd.sparkContext){ + new MapPartitionsRDD( + rdd, + (_: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), + isOrderSensitive = isOrderSensitive) { + override protected def getOutputDeterministicLevel = + DeterministicLevel.INDETERMINATE + } + } + } else { + rdd.mapPartitionsWithIndexInternal(f, isOrderSensitive = isOrderSensitive) + } + + /** * Create a customized [[ShuffleWriteProcessor]] for SQL which wrap the default metrics reporter * with [[SQLShuffleWriteMetricsReporter]] as new reporter for [[ShuffleWriteProcessor]]. From b9bd30bf7b01bfef4dd32cc7099e0c1c960b093f Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 20 Mar 2025 15:37:02 -0700 Subject: [PATCH 45/48] SPARK-51016. Implemented review feedback from @squito and @mridulm --- .../spark/sql/execution/exchange/ShuffleExchangeExec.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 e5a3e935c691..4915e9a703b3 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 @@ -19,8 +19,11 @@ package org.apache.spark.sql.execution.exchange import java.util.concurrent.atomic.AtomicReference import java.util.function.Supplier + import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future, Promise} +import scala.reflect.ClassTag + import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.rdd.{DeterministicLevel, MapPartitionsRDD, RDD, RDDOperationScope} @@ -41,7 +44,6 @@ import org.apache.spark.util.{MutablePair, ThreadUtils} import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} import org.apache.spark.util.random.XORShiftRandom -import scala.reflect.ClassTag /** * Common trait for all shuffle exchange implementations to facilitate pattern matching. @@ -504,7 +506,7 @@ object ShuffleExchangeExec { f: (Int, Iterator[T]) => Iterator[U], isInDeterminate: Boolean, isOrderSensitive: Boolean): RDD[U] = if (isInDeterminate) { - RDDOperationScope.withScope(rdd.sparkContext){ + RDDOperationScope.withScope(rdd.sparkContext) { new MapPartitionsRDD( rdd, (_: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), From f7a6721de4933fc4c5428782f6a0c5b7fbf69865 Mon Sep 17 00:00:00 2001 From: ashahid Date: Thu, 20 Mar 2025 15:39:14 -0700 Subject: [PATCH 46/48] SPARK-51016. Implemented review feedback from @squito and @mridulm --- .../spark/sql/execution/exchange/ShuffleExchangeExec.scala | 3 --- 1 file changed, 3 deletions(-) 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 4915e9a703b3..2b68b2feb07e 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 @@ -493,9 +493,6 @@ object ShuffleExchangeExec { rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), serializer, - None, - None, - false, shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics)) dependency From 914123f81fdd7b56ab8044a4e90a2c5acf9bb4c0 Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 2 Apr 2025 20:40:13 -0700 Subject: [PATCH 47/48] SPARK-51272. Fixed the tests code as per feedback to use direct dependencies of ResultStage on two Shuffle Stages, instead of the usual transitive dependency of shuffle stage2 on shuffle stage1 --- .../spark/scheduler/DAGSchedulerSuite.scala | 78 ++++++++++++------- 1 file changed, 49 insertions(+), 29 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 5d4f759d83bb..91adf3d2d233 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3171,24 +3171,39 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti private def constructTwoStages( stage1InDeterminate: Boolean, - stage2InDeterminate: Boolean): (Int, Int) = { + stage2InDeterminate: Boolean, + isDependencyBetweenStagesTransitive: Boolean = true): (Int, Int) = { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = stage1InDeterminate) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) val shuffleId1 = shuffleDep1.shuffleId - val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker, - indeterminate = stage2InDeterminate) + val shuffleMapRdd2 = if (isDependencyBetweenStagesTransitive) { + new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker, + indeterminate = stage2InDeterminate) + } else { + new MyRDD(sc, 2, Nil, tracker = mapOutputTracker, indeterminate = stage2InDeterminate) + } val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) val shuffleId2 = shuffleDep2.shuffleId - val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) + + val finalRdd = if (isDependencyBetweenStagesTransitive) { + new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) + } else { + new MyRDD(sc, 2, List(shuffleDep1, shuffleDep2), tracker = mapOutputTracker) + } submit(finalRdd, Array(0, 1)) + val stageId1 = this.scheduler.shuffleIdToMapStage(shuffleId1).id // Finish the first shuffle map stage. - completeShuffleMapStageSuccessfully(0, 0, 2) - assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) - + completeShuffleMapStageSuccessfully(stageId1, 0, 2) + import org.scalatest.concurrent.Eventually._ + import org.scalatest.matchers.should.Matchers._ + import org.scalatest.time.SpanSugar._ + eventually(timeout(1.minutes), interval(500.milliseconds)) { + mapOutputTracker.findMissingPartitions(shuffleId1) should equal(Some(Nil)) + } (shuffleId1, shuffleId2) } @@ -3211,10 +3226,14 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val numPartitions = 2 // The first shuffle stage is completed by the below function itself which creates two - // indeterminate stages. + // stages. val (shuffleId1, shuffleId2) = constructTwoStages( - stage1InDeterminate = false, stage2InDeterminate = true) - completeShuffleMapStageSuccessfully(shuffleId2, 0, numPartitions) + stage1InDeterminate = false, + stage2InDeterminate = true, + isDependencyBetweenStagesTransitive = false) + val shuffleStage1 = this.scheduler.shuffleIdToMapStage(shuffleId1) + val shuffleStage2 = this.scheduler.shuffleIdToMapStage(shuffleId2) + completeShuffleMapStageSuccessfully(shuffleStage2.id, 0, numPartitions) val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] val activeJob = resultStage.activeJob assert(activeJob.isDefined) @@ -3231,20 +3250,19 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti taskSets(2).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), null)) - val shuffleStage1 = this.scheduler.shuffleIdToMapStage(shuffleId1) - val shuffleStage2 = this.scheduler.shuffleIdToMapStage(shuffleId2) + import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ eventually(timeout(3.minutes), interval(500.milliseconds)) { shuffleStage1.latestInfo.attemptNumber() should equal(1) } - completeShuffleMapStageSuccessfully(0, 1, numPartitions) + completeShuffleMapStageSuccessfully(shuffleStage1.id, 1, numPartitions) eventually(timeout(3.minutes), interval(500.milliseconds)) { shuffleStage2.latestInfo.attemptNumber() should equal(1) } - completeShuffleMapStageSuccessfully(1, 1, numPartitions) + completeShuffleMapStageSuccessfully(shuffleStage2.id, 1, numPartitions) eventually(timeout(3.minutes), interval(500.milliseconds)) { resultStage.latestInfo.attemptNumber() should equal(1) } @@ -3261,9 +3279,13 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti val numPartitions = 2 // The first shuffle stage is completed by the below function itself which creates two // stages. - val (detShuffleId1, indetShuffleId2) = constructTwoStages(stage1InDeterminate = false, - stage2InDeterminate = true) - completeShuffleMapStageSuccessfully(indetShuffleId2, 0, numPartitions) + val (detShuffleId1, indetShuffleId2) = constructTwoStages( + stage1InDeterminate = false, + stage2InDeterminate = true, + isDependencyBetweenStagesTransitive = false) + val detShuffleStage1 = this.scheduler.shuffleIdToMapStage(detShuffleId1) + val inDetshuffleStage2 = this.scheduler.shuffleIdToMapStage(indetShuffleId2) + completeShuffleMapStageSuccessfully(inDetshuffleStage2.id, 0, numPartitions) assert(mapOutputTracker.findMissingPartitions(indetShuffleId2) === Some(Seq.empty)) val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] val activeJob = resultStage.activeJob @@ -3288,15 +3310,14 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti taskSets(2).tasks(0), FetchFailed(makeBlockManagerId("hostA"), detShuffleId1, 0L, 0, 0, "ignored"), null)) - val detShuffleStage1 = this.scheduler.shuffleIdToMapStage(detShuffleId1) - val inDetshuffleStage2 = this.scheduler.shuffleIdToMapStage(indetShuffleId2) + import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ eventually(timeout(3.minutes), interval(500.milliseconds)) { detShuffleStage1.latestInfo.attemptNumber() should equal(1) } - completeShuffleMapStageSuccessfully(0, 1, numPartitions) + completeShuffleMapStageSuccessfully(detShuffleStage1.id, 1, numPartitions) // Though the inDetShuffleStage2 has not suffered any loss, but source code of DagScheduler // has code to remove shuffleoutputs based on the lost BlockManager , which in this case will @@ -3308,7 +3329,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti } org.scalatest.Assertions.assert(inDetshuffleStage2.latestInfo.numTasks == 2) org.scalatest.Assertions.assert(inDetshuffleStage2.findMissingPartitions().size == 2) - completeShuffleMapStageSuccessfully(1, 1, numPartitions) + completeShuffleMapStageSuccessfully(inDetshuffleStage2.id, 1, numPartitions) eventually(timeout(3.minutes), interval(500.milliseconds)) { resultStage.latestInfo.attemptNumber() should equal(1) } @@ -3323,27 +3344,26 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // The first shuffle stage is completed by the below function itself which creates two // indeterminate stages. val (shuffleId1, shuffleId2) = constructTwoStages( - stage1InDeterminate = false, stage2InDeterminate = true) + stage1InDeterminate = false, + stage2InDeterminate = true, + isDependencyBetweenStagesTransitive = false + ) // This will trigger the resubmit failed stage and in before adding resubmit message to the // queue, a successful partition completion event will arrive. runEvent( makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId2, 0L, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId2, 0L, 0, 0, "ignored"), null)) - // run completion task for first stage as the Fetch Failed for second shuffle stage is sharing - // the block Id - completeShuffleMapStageSuccessfully(0, 1, numPartitions) - - val shuffleStage2 = scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage] + val shuffleStage2 = scheduler.shuffleIdToMapStage(shuffleId2) import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._ - eventually(timeout(3.minutes), interval(500.milliseconds)) { + eventually(timeout(30.seconds), interval(500.milliseconds)) { shuffleStage2.latestInfo.attemptNumber() should equal(1) } org.scalatest.Assertions.assert(shuffleStage2.findMissingPartitions().size == numPartitions) From 00a4aadb8cfce30f2234453c64b9ca46c60fa07f Mon Sep 17 00:00:00 2001 From: ashahid Date: Wed, 2 Apr 2025 23:17:21 -0700 Subject: [PATCH 48/48] SPARK-51272. Fixed the race in test causing inconsistent pass --- .../spark/scheduler/DAGSchedulerSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 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 91adf3d2d233..0f00ce302962 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3222,7 +3222,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti test("SPARK-51272: retry all the partitions of result stage, if the first result task" + " has failed and failing ShuffleMap stage is inDeterminate") { this.dagSchedulerInterceptor = createDagInterceptorForSpark51272( - () => taskSets(2).tasks(1), "RELEASE_LATCH") + () => taskSets.find(_.shuffleId.isEmpty).get.tasks(1), "RELEASE_LATCH") val numPartitions = 2 // The first shuffle stage is completed by the below function itself which creates two @@ -3247,7 +3247,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // of result tasks (2 tasks in total), only some (1 task) get retried runEvent( makeCompletionEvent( - taskSets(2).tasks(0), + taskSets.find(_.stageId == resultStage.id).get.tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), null)) @@ -3274,7 +3274,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti " has failed with failing ShuffleStage determinate but result stage has another ShuffleStage" + " which is indeterminate") { this.dagSchedulerInterceptor = createDagInterceptorForSpark51272( - () => taskSets(2).tasks(1), "RELEASE_LATCH") + () => taskSets.find(_.shuffleId.isEmpty).get.tasks(1), "RELEASE_LATCH") val numPartitions = 2 // The first shuffle stage is completed by the below function itself which creates two @@ -3307,7 +3307,7 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // runEvent( makeCompletionEvent( - taskSets(2).tasks(0), + taskSets.find(_.shuffleId.isEmpty).get.tasks(0), FetchFailed(makeBlockManagerId("hostA"), detShuffleId1, 0L, 0, 0, "ignored"), null)) @@ -3340,7 +3340,8 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti " has failed and failing ShuffleMap stage is inDeterminate") { val numPartitions = 2 this.dagSchedulerInterceptor = createDagInterceptorForSpark51272( - () => taskSets(1).tasks(1), makeMapStatus(host = "hostZZZ", reduces = numPartitions)) + () => taskSets.filter(_.shuffleId.isDefined).maxBy(_.shuffleId.get).tasks(1), + makeMapStatus(host = "hostZZZ", reduces = numPartitions)) // The first shuffle stage is completed by the below function itself which creates two // indeterminate stages. val (shuffleId1, shuffleId2) = constructTwoStages( @@ -3350,15 +3351,13 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti ) // This will trigger the resubmit failed stage and in before adding resubmit message to the // queue, a successful partition completion event will arrive. - runEvent( makeCompletionEvent( - taskSets(1).tasks(0), + taskSets.filter(_.shuffleId.isDefined).maxBy(_.shuffleId.get).tasks(0), FetchFailed(makeBlockManagerId("hostB"), shuffleId2, 0L, 0, 0, "ignored"), null)) val shuffleStage2 = scheduler.shuffleIdToMapStage(shuffleId2) - import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.should.Matchers._ import org.scalatest.time.SpanSugar._