diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 573608c4327e0..2751d7b3b2e04 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 80db818b77e42..978d80db67c72 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -2079,6 +2079,7 @@ abstract class RDD[T: ClassTag]( private final lazy val _outputDeterministicLevel: DeterministicLevel.Value = getOutputDeterministicLevel + /** * Returns the deterministic level of this RDD's output. Please refer to [[DeterministicLevel]] * for the definition. @@ -2105,6 +2106,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/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py index 3f6831b600678..bda05017e135f 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 a5b6a17c6ae64..b945fb4ecc536 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 2af6a1ba84ec8..421c2fd190a4f 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 184f5a2a9485d..2347410d45377 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 bf1c930c0bd0b..983810a5fdaec 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 31a3f53eb7191..2e6261de40401 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/sql/execution/ProjectedOrderingAndPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala index ec13d48d45f84..395c03b81055f 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 {