From 5d68f37190b1368cadca796937357c0e02d5be57 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 9 Jun 2021 12:26:29 -0700 Subject: [PATCH 01/29] wip --- .../sql/tests/test_pandas_cogrouped_map.py | 1 + .../plans/physical/partitioning.scala | 111 ++- .../sql/catalyst/DistributionSuite.scala | 221 ++++- .../DisableUnnecessaryBucketedScan.scala | 4 +- .../exchange/EnsureRequirements.scala | 30 + .../sql/execution/joins/ShuffledJoin.scala | 4 +- .../apache/spark/sql/execution/objects.scala | 2 +- .../StreamingSymmetricHashJoinExec.scala | 4 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 235 +++-- .../q47.sf100/simplified.txt | 169 ++-- .../approved-plans-v1_4/q57.sf100/explain.txt | 235 +++-- .../q57.sf100/simplified.txt | 169 ++-- .../approved-plans-v2_7/q47.sf100/explain.txt | 235 +++-- .../q47.sf100/simplified.txt | 169 ++-- .../q51a.sf100/explain.txt | 508 ++++++----- .../q51a.sf100/simplified.txt | 206 +++-- .../approved-plans-v2_7/q57.sf100/explain.txt | 235 +++-- .../q57.sf100/simplified.txt | 169 ++-- .../approved-plans-v2_7/q64/explain.txt | 838 +++++++++--------- .../approved-plans-v2_7/q64/simplified.txt | 524 ++++++----- .../exchange/EnsureRequirementsSuite.scala | 29 + .../spark/sql/sources/BucketedReadSuite.scala | 3 +- .../sql/streaming/StreamingJoinSuite.scala | 2 +- 23 files changed, 2117 insertions(+), 1986 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py index 218e7cc77cf74..c01f60bb388d1 100644 --- a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py @@ -238,6 +238,7 @@ def right_assign_key(key, l, r): left.groupby("id") .cogroup(right.groupby("id")) .applyInPandas(right_assign_key, "id long, k int, v int, key long") + .sort(['id']) .toPandas() ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index fb7089c6aec9f..b79288438753b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.plans.physical +import scala.collection.mutable + import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{DataType, IntegerType} @@ -87,31 +89,6 @@ case class ClusteredDistribution( } } -/** - * Represents data where tuples have been clustered according to the hash of the given - * `expressions`. The hash function is defined as `HashPartitioning.partitionIdExpression`, so only - * [[HashPartitioning]] can satisfy this distribution. - * - * This is a strictly stronger guarantee than [[ClusteredDistribution]]. Given a tuple and the - * number of partitions, this distribution strictly requires which partition the tuple should be in. - */ -case class HashClusteredDistribution( - expressions: Seq[Expression], - requiredNumPartitions: Option[Int] = None) extends Distribution { - require( - expressions != Nil, - "The expressions for hash of a HashClusteredDistribution should not be Nil. " + - "An AllTuples should be used to represent a distribution that only has " + - "a single partition.") - - override def createPartitioning(numPartitions: Int): Partitioning = { - assert(requiredNumPartitions.isEmpty || requiredNumPartitions.get == numPartitions, - s"This HashClusteredDistribution requires ${requiredNumPartitions.get} partitions, but " + - s"the actual number of partitions is $numPartitions.") - HashPartitioning(expressions, numPartitions) - } -} - /** * Represents data where tuples have been ordered according to the `ordering` * [[Expression Expressions]]. Its requirement is defined as the following: @@ -171,6 +148,24 @@ trait Partitioning { required.requiredNumPartitions.forall(_ == numPartitions) && satisfies0(required) } + /** + * Returns true iff this partitioning is compatible with `other`. If two [[Partitioning]]s can + * satisfy their respective required distribution (via [[satisfies]]), and are compatible with + * each other, then their partitions are considered to be co-partitioned, which will allow Spark + * to eliminate data shuffle whenever necessary. + * + * Note: implementor should make sure the method satisfies the equivalence relation, that is, + * the implementation should be reflexive, symmetric and transitive. + */ + final def isCompatibleWith( + distribution: Distribution, + other: Partitioning, + otherDistribution: Distribution): Boolean = other match { + case PartitioningCollection(others) => + others.exists(_.isCompatibleWith(otherDistribution, this, distribution)) + case _ => isCompatibleWith0(distribution, other, otherDistribution) + } + /** * The actual method that defines whether this [[Partitioning]] can satisfy the given * [[Distribution]], after the `numPartitions` check. @@ -184,6 +179,15 @@ trait Partitioning { case AllTuples => numPartitions == 1 case _ => false } + + /** + * The actual method that defines whether this [[Partitioning]] is compatible with `other`. In + * default this always return false. + */ + protected def isCompatibleWith0( + distribution: Distribution, + other: Partitioning, + otherDistribution: Distribution): Boolean = false } case class UnknownPartitioning(numPartitions: Int) extends Partitioning @@ -202,6 +206,14 @@ case object SinglePartition extends Partitioning { case _: BroadcastDistribution => false case _ => true } + + override def isCompatibleWith0( + distribution: Distribution, + other: Partitioning, + otherDistribution: Distribution): Boolean = other match { + case SinglePartition => true + case _ => false + } } /** @@ -219,10 +231,6 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) override def satisfies0(required: Distribution): Boolean = { super.satisfies0(required) || { required match { - case h: HashClusteredDistribution => - expressions.length == h.expressions.length && expressions.zip(h.expressions).forall { - case (l, r) => l.semanticEquals(r) - } case ClusteredDistribution(requiredClustering, _) => expressions.forall(x => requiredClustering.exists(_.semanticEquals(x))) case _ => false @@ -230,6 +238,45 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } + override def isCompatibleWith0( + distribution: Distribution, + other: Partitioning, + otherDistribution: Distribution): Boolean = (distribution, otherDistribution) match { + case (thisDist: ClusteredDistribution, thatDist: ClusteredDistribution) => + // For each expression in the `HashPartitioning` that has occurrences in + // `ClusteredDistribution`, returns a mapping from its index in the partitioning to the + // indexes where it appears in the distribution. + // For instance, if `partitioning` is `[a, b]` and `distribution is `[a, a, b]`, then the + // result mapping could be `{ 0 -> (0, 1), 1 -> (2) }`. + def indexMap( + distribution: ClusteredDistribution, + partitioning: HashPartitioning): mutable.Map[Int, mutable.BitSet] = { + val result = mutable.Map.empty[Int, mutable.BitSet] + val expressionToIndex = partitioning.expressions.zipWithIndex.toMap + distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyIdx) => + expressionToIndex.find { case (partKey, _) => partKey.semanticEquals(distKey) }.forall { + case (_, partIdx) => + result.getOrElseUpdate(partIdx, mutable.BitSet.empty).add(distKeyIdx) + } + } + result + } + + other match { + case that @ HashPartitioning(_, _) => + // we need to check: + // 1. both partitioning have the same number of expressions + // 2. each corresponding expression in both partitioning is used in the same positions + // of the corresponding distribution. + this.expressions.length == that.expressions.length && + indexMap(thisDist, this) == indexMap(thatDist, that) + case _ => + false + } + case _ => + false + } + /** * Returns an expression that will produce a valid partition ID(i.e. non-negative and is less * than numPartitions) based on hashing expressions. @@ -330,6 +377,12 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def satisfies0(required: Distribution): Boolean = partitionings.exists(_.satisfies(required)) + override def isCompatibleWith0( + distribution: Distribution, + other: Partitioning, + otherDistribution: Distribution): Boolean = + partitionings.exists(_.isCompatibleWith(distribution, other, otherDistribution)) + override def toString: String = { partitionings.map(_.toString).mkString("(", " or ", ")") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 02b25a2fa1814..8b7f9ac41c79d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -41,6 +41,36 @@ class DistributionSuite extends SparkFunSuite { } } + protected def checkCompatible( + left: Partitioning, + right: Partitioning, + compatible: Boolean, + leftDistribution: Distribution = UnspecifiedDistribution, + rightDistribution: Distribution = UnspecifiedDistribution): Unit = { + val actual = left.isCompatibleWith(leftDistribution, right, rightDistribution) + if (actual != compatible) { + fail( + s""" + |== Left Partitioning == + |$left + |== Right Partitioning == + |$right + |== Is left partitioning compatible with right partitioning? == + |Expected $compatible but got $actual + |""".stripMargin) + } + } + + protected def checkPartitionCollectionCompatible( + left: Partitioning, + right: Partitioning, + compatible: Boolean, + leftDistribution: Distribution = UnspecifiedDistribution, + rightDistribution: Distribution = UnspecifiedDistribution): Unit = { + checkCompatible(left, right, compatible, leftDistribution, rightDistribution) + checkCompatible(right, left, compatible, rightDistribution, leftDistribution) + } + test("UnspecifiedDistribution and AllTuples") { // all partitioning can satisfy UnspecifiedDistribution checkSatisfied( @@ -133,11 +163,6 @@ class DistributionSuite extends SparkFunSuite { ClusteredDistribution(Seq($"a", $"b", $"c")), true) - checkSatisfied( - SinglePartition, - HashClusteredDistribution(Seq($"a", $"b", $"c")), - true) - checkSatisfied( SinglePartition, OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), @@ -172,23 +197,6 @@ class DistributionSuite extends SparkFunSuite { ClusteredDistribution(Seq($"d", $"e")), false) - // HashPartitioning can satisfy HashClusteredDistribution iff its hash expressions are exactly - // same with the required hash clustering expressions. - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - HashClusteredDistribution(Seq($"a", $"b", $"c")), - true) - - checkSatisfied( - HashPartitioning(Seq($"c", $"b", $"a"), 10), - HashClusteredDistribution(Seq($"a", $"b", $"c")), - false) - - checkSatisfied( - HashPartitioning(Seq($"a", $"b"), 10), - HashClusteredDistribution(Seq($"a", $"b", $"c")), - false) - // HashPartitioning cannot satisfy OrderedDistribution checkSatisfied( HashPartitioning(Seq($"a", $"b", $"c"), 10), @@ -269,12 +277,6 @@ class DistributionSuite extends SparkFunSuite { RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), ClusteredDistribution(Seq($"c", $"d")), false) - - // RangePartitioning cannot satisfy HashClusteredDistribution - checkSatisfied( - RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), - HashClusteredDistribution(Seq($"a", $"b", $"c")), - false) } test("Partitioning.numPartitions must match Distribution.requiredNumPartitions to satisfy it") { @@ -283,24 +285,167 @@ class DistributionSuite extends SparkFunSuite { ClusteredDistribution(Seq($"a", $"b", $"c"), Some(10)), false) - checkSatisfied( - SinglePartition, - HashClusteredDistribution(Seq($"a", $"b", $"c"), Some(10)), - false) - checkSatisfied( HashPartitioning(Seq($"a", $"b", $"c"), 10), ClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), false) - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - HashClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), - false) - checkSatisfied( RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), ClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), false) } + + test("Compatibility: SinglePartition and HashPartitioning") { + checkCompatible( + SinglePartition, + SinglePartition, + compatible = true) + + checkCompatible( + HashPartitioning(Seq($"a", $"b", $"c"), 4), + HashPartitioning(Seq($"a", $"b", $"c"), 4), + compatible = true, + ClusteredDistribution(Seq($"a", $"b", $"c")), + ClusteredDistribution(Seq($"a", $"b", $"c"))) + + checkCompatible( + HashPartitioning(Seq($"a", $"c"), 4), + HashPartitioning(Seq($"a", $"c"), 8), + compatible = true, + ClusteredDistribution(Seq($"a", $"b", $"c")), + ClusteredDistribution(Seq($"a", $"b", $"c"))) + + checkCompatible( + HashPartitioning(Seq($"a", $"b"), 4), + HashPartitioning(Seq($"b", $"a"), 8), + compatible = true, + ClusteredDistribution(Seq($"a", $"b")), + ClusteredDistribution(Seq($"b", $"a"))) + + checkCompatible( + HashPartitioning(Seq($"a", $"b"), 4), + HashPartitioning(Seq($"b", $"a"), 8), + compatible = true, + ClusteredDistribution(Seq($"c", $"a", $"b")), + ClusteredDistribution(Seq($"d", $"b", $"a"))) + + checkCompatible( + HashPartitioning(Seq($"a", $"b"), 4), + HashPartitioning(Seq($"a", $"b"), 8), + compatible = true, + ClusteredDistribution(Seq($"a", $"a", $"b")), + ClusteredDistribution(Seq($"a", $"a", $"b"))) + + // negative cases + + checkCompatible( + HashPartitioning(Seq($"a"), 4), + HashPartitioning(Seq($"b"), 4), + compatible = false, + ClusteredDistribution(Seq($"a", $"b")), + ClusteredDistribution(Seq($"a", $"b"))) + + checkCompatible( + HashPartitioning(Seq($"a", $"b"), 4), + HashPartitioning(Seq($"b", $"b"), 4), + compatible = false, + ClusteredDistribution(Seq($"a", $"b")), + ClusteredDistribution(Seq($"b", $"a"))) + + checkCompatible( + HashPartitioning(Seq($"a", $"b"), 4), + HashPartitioning(Seq($"a", $"b"), 4), + compatible = false, + ClusteredDistribution(Seq($"a", $"a", $"b")), + ClusteredDistribution(Seq($"a", $"b", $"b"))) + } + + test("Compatibility: PartitionCollection") { + checkPartitionCollectionCompatible( + HashPartitioning(Seq($"a"), 4), + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), + compatible = true, + ClusteredDistribution(Seq($"a")), + ClusteredDistribution(Seq($"a"))) + + checkPartitionCollectionCompatible( + HashPartitioning(Seq($"a"), 4), + PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4), HashPartitioning(Seq($"a"), 4))), + compatible = true, + ClusteredDistribution(Seq($"a")), + ClusteredDistribution(Seq($"a", $"b"))) + + checkPartitionCollectionCompatible( + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), + compatible = true, + ClusteredDistribution(Seq($"a")), + ClusteredDistribution(Seq($"a"))) + + checkPartitionCollectionCompatible( + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), + compatible = true, + ClusteredDistribution(Seq($"a", $"b")), + ClusteredDistribution(Seq($"a", $"b"))) + + checkPartitionCollectionCompatible( + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), + PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4), HashPartitioning(Seq($"a"), 4))), + compatible = true, + ClusteredDistribution(Seq($"a", $"b")), + ClusteredDistribution(Seq($"a", $"b"))) + + checkPartitionCollectionCompatible( + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), + compatible = true, + ClusteredDistribution(Seq($"a")), + ClusteredDistribution(Seq($"a", $"b"))) + + checkPartitionCollectionCompatible( + PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4), HashPartitioning(Seq($"a"), 4))), + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"c"), 4))), + compatible = true, + ClusteredDistribution(Seq($"b", $"a")), + ClusteredDistribution(Seq($"a", $"c"))) + + // negative cases + + checkPartitionCollectionCompatible( + HashPartitioning(Seq($"a"), 4), + PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4))), + compatible = false, + ClusteredDistribution(Seq($"a", $"b")), + ClusteredDistribution(Seq($"a", $"b"))) + + checkCompatible( + PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), + PartitioningCollection(Seq(HashPartitioning(Seq($"c"), 4))), + compatible = false, + ClusteredDistribution(Seq($"a", $"b", $"c")), + ClusteredDistribution(Seq($"a", $"b", $"c"))) + } + + test("Compatibility: Others") { + val partitionings: Seq[Partitioning] = Seq(UnknownPartitioning(1), + BroadcastPartitioning(IdentityBroadcastMode), + RoundRobinPartitioning(10), + RangePartitioning(Seq($"a".asc), 10), + PartitioningCollection(Seq(UnknownPartitioning(1))) + ) + + for (i <- partitionings.indices) { + for (j <- partitionings.indices) { + checkCompatible(partitionings(i), partitionings(j), compatible = false) + } + } + + // should always return false when comparing with `HashPartitioning` or `SinglePartition` + partitionings.foreach { p => + checkCompatible(p, HashPartitioning(Seq($"a"), 10), compatible = false) + checkCompatible(p, SinglePartition, compatible = false) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala index 5bd70c61fd1fd..479bc21e5e6c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.bucketing -import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashClusteredDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.aggregate.BaseAggregateExec @@ -120,7 +120,7 @@ object DisableUnnecessaryBucketedScan extends Rule[SparkPlan] { private def hasInterestingPartition(plan: SparkPlan): Boolean = { plan.requiredChildDistribution.exists { - case _: ClusteredDistribution | _: HashClusteredDistribution | AllTuples => true + case _: ClusteredDistribution | AllTuples => true case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index e73b48c4e7de0..d8d76910d2c6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -73,6 +73,36 @@ case class EnsureRequirements( case _ => true }.map(_._2) + // Assuming equivalence relation in partitioning compatibility check + val allCompatible = childrenIndexes + .map(i => (children(i).outputPartitioning, requiredChildDistributions(i))) + .sliding(2).map { + case Seq(_) => true + case Seq((ap, ad), (bp, bd)) => ap.isCompatibleWith(ad, bp, bd) + }.forall(_ == true) + + if (!allCompatible) { + // insert shuffle for all children that are not compatible + children = children.zip(requiredChildDistributions).zipWithIndex.map { + case ((child, _), idx) if !childrenIndexes.contains(idx) => + child + case ((child, dist), _) => + val numPartitions = dist.requiredNumPartitions.getOrElse(conf.numShufflePartitions) + val defaultPartitioning = dist.createPartitioning(numPartitions) + // check if the child's partitioning is already the same as default partitioning, and + // skip the shuffle if so. + // TODO: we should find the "least common" partitioning for all children and use that + if (!child.outputPartitioning.isCompatibleWith(dist, defaultPartitioning, dist)) { + child match { + case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(defaultPartitioning, c, so) + case _ => ShuffleExchangeExec(defaultPartitioning, child) + } + } else { + child + } + } + } + val childrenNumPartitions = childrenIndexes.map(children(_).outputPartitioning.numPartitions).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala index 794807fd3b463..7c4628c8576c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, LeftExistence, LeftOuter, RightOuter} -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution, Partitioning, PartitioningCollection, UnknownPartitioning, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning, PartitioningCollection, UnknownPartitioning, UnspecifiedDistribution} /** * Holds common logic for join operators by shuffling two child relations @@ -40,7 +40,7 @@ trait ShuffledJoin extends JoinCodegenSupport { // partitioning doesn't satisfy `HashClusteredDistribution`. UnspecifiedDistribution :: UnspecifiedDistribution :: Nil } else { - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index fa46f75abe8f3..869d3fe979010 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -628,7 +628,7 @@ case class CoGroupExec( right: SparkPlan) extends BinaryExecNode with ObjectProducerExec { override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftGroup) :: HashClusteredDistribution(rightGroup) :: Nil + ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil override def requiredChildOrdering: Seq[Seq[SortOrder]] = leftGroup.map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 616ae08a26e6b..74b82451e029f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -185,8 +185,8 @@ case class StreamingSymmetricHashJoinExec( val nullRight = new GenericInternalRow(right.output.map(_.withNullability(true)).length) override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftKeys, stateInfo.map(_.numPartitions)) :: - HashClusteredDistribution(rightKeys, stateInfo.map(_.numPartitions)) :: Nil + ClusteredDistribution(leftKeys, stateInfo.map(_.numPartitions)) :: + ClusteredDistribution(rightKeys, stateInfo.map(_.numPartitions)) :: Nil override def output: Seq[Attribute] = joinType match { case _: InnerLike => left.output ++ right.output diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index 529b9c8282db5..44a956471b61e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -1,56 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- * HashAggregate (24) - : : +- Exchange (23) - : : +- * HashAggregate (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.store (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (41) - : +- Exchange (40) - : +- * Project (39) - : +- Window (38) - : +- * Sort (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * Sort (49) - +- Exchange (48) - +- * Project (47) - +- Window (46) - +- * Sort (45) - +- ReusedExchange (44) +TakeOrderedAndProject (49) ++- * Project (48) + +- * SortMergeJoin Inner (47) + :- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (32) + : : +- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Sort (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.store (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (39) + : +- * Project (38) + : +- Window (37) + : +- * Sort (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- ReusedExchange (33) + +- * Sort (46) + +- * Project (45) + +- Window (44) + +- * Sort (43) + +- ReusedExchange (42) (1) Scan parquet default.store_sales @@ -68,7 +65,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) -(4) ReusedExchange [Reuses operator id: 56] +(4) ReusedExchange [Reuses operator id: 53] Output [3]: [d_date_sk#6, d_year#7, d_moy#8] (5) BroadcastHashJoin [codegen id : 3] @@ -192,118 +189,106 @@ Condition : ((isnotnull(avg_monthly_sales#26) AND (avg_monthly_sales#26 > 0.0000 Output [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25] Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, _w0#23, rn#25, avg_monthly_sales#26] -(32) Exchange -Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25] -Arguments: hashpartitioning(i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25, 5), ENSURE_REQUIREMENTS, [id=#27] - -(33) Sort [codegen id : 12] +(32) Sort [codegen id : 11] Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25] Arguments: [i_category#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST, s_company_name#11 ASC NULLS FIRST, rn#25 ASC NULLS FIRST], false, 0 -(34) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] +(33) ReusedExchange [Reuses operator id: 23] +Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] -(35) HashAggregate [codegen id : 20] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] -Keys [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(ss_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#21] -Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#21,17,2) AS sum_sales#22] +(34) HashAggregate [codegen id : 19] +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] +Keys [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32] +Functions [1]: [sum(UnscaledValue(ss_sales_price#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#34))#21] +Results [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#34))#21,17,2) AS sum_sales#22] -(36) Exchange -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(35) Exchange +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22] +Arguments: hashpartitioning(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, 5), ENSURE_REQUIREMENTS, [id=#35] -(37) Sort [codegen id : 21] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 20] +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST], false, 0 -(38) Window -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +(37) Window +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#27, i_brand#28, s_store_name#29, s_company_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#22 AS sum_sales#38, rn#37] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22, rn#37] +(38) Project [codegen id : 21] +Output [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#22 AS sum_sales#37, rn#36] +Input [8]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22, rn#36] -(40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] +(39) Sort [codegen id : 21] +Input [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, (rn#36 + 1) ASC NULLS FIRST], false, 0 -(41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 24] +(40) SortMergeJoin [codegen id : 22] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1)] +Right keys [5]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, (rn#36 + 1)] Join condition: None -(43) Project [codegen id : 24] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] - -(44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] +(41) Project [codegen id : 22] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36] -(45) Sort [codegen id : 33] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +(42) ReusedExchange [Reuses operator id: 35] +Output [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22] -(46) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] +(43) Sort [codegen id : 31] +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 -(47) Project [codegen id : 34] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#22 AS sum_sales#47, rn#46] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22, rn#46] +(44) Window +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, s_store_name#40, s_company_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] -(48) Exchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] +(45) Project [codegen id : 32] +Output [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#22 AS sum_sales#45, rn#44] +Input [8]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22, rn#44] -(49) Sort [codegen id : 35] -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 32] +Input [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, (rn#44 - 1) ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 36] +(47) SortMergeJoin [codegen id : 33] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1)] +Right keys [5]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, (rn#44 - 1)] Join condition: None -(51) Project [codegen id : 36] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#38 AS psum#49, sum_sales#47 AS nsum#50] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] +(48) Project [codegen id : 33] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#46, sum_sales#45 AS nsum#47] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] -(52) TakeOrderedAndProject -Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] +(49) TakeOrderedAndProject +Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (56) -+- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet default.date_dim (53) +BroadcastExchange (53) ++- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(53) Scan parquet default.date_dim +(50) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -(55) Filter [codegen id : 1] +(52) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6)) -(56) BroadcastExchange +(53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt index 07c75d91ca3cf..aa2346cacaf2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt @@ -1,104 +1,95 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (36) + WholeStageCodegen (33) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,rn] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #1 - WholeStageCodegen (11) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (10) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (9) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #2 - WholeStageCodegen (8) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3 - WholeStageCodegen (7) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (10) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (9) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (8) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk,s_store_name,s_company_name] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk,s_store_name,s_company_name] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_company_name] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk,i_category,i_brand] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_category] + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk,i_category,i_brand] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (21) Sort [i_category,i_brand,s_store_name,s_company_name,rn] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 - WholeStageCodegen (22) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (21) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #9 - WholeStageCodegen (20) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (20) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (19) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - WholeStageCodegen (35) + WholeStageCodegen (32) Sort [i_category,i_brand,s_store_name,s_company_name,rn] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 - WholeStageCodegen (34) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (33) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #9 + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (31) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index ef8d64cee2c4a..ad356d44af668 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -1,56 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- * HashAggregate (24) - : : +- Exchange (23) - : : +- * HashAggregate (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.call_center (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (41) - : +- Exchange (40) - : +- * Project (39) - : +- Window (38) - : +- * Sort (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * Sort (49) - +- Exchange (48) - +- * Project (47) - +- Window (46) - +- * Sort (45) - +- ReusedExchange (44) +TakeOrderedAndProject (49) ++- * Project (48) + +- * SortMergeJoin Inner (47) + :- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (32) + : : +- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Sort (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.catalog_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.call_center (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (39) + : +- * Project (38) + : +- Window (37) + : +- * Sort (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- ReusedExchange (33) + +- * Sort (46) + +- * Project (45) + +- Window (44) + +- * Sort (43) + +- ReusedExchange (42) (1) Scan parquet default.catalog_sales @@ -68,7 +65,7 @@ Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk#4] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_call_center_sk#1)) -(4) ReusedExchange [Reuses operator id: 56] +(4) ReusedExchange [Reuses operator id: 53] Output [3]: [d_date_sk#6, d_year#7, d_moy#8] (5) BroadcastHashJoin [codegen id : 3] @@ -192,118 +189,106 @@ Condition : ((isnotnull(avg_monthly_sales#25) AND (avg_monthly_sales#25 > 0.0000 Output [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24] Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, _w0#22, rn#24, avg_monthly_sales#25] -(32) Exchange -Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24] -Arguments: hashpartitioning(i_category#15, i_brand#14, cc_name#10, rn#24, 5), ENSURE_REQUIREMENTS, [id=#26] - -(33) Sort [codegen id : 12] +(32) Sort [codegen id : 11] Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24] Arguments: [i_category#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST, rn#24 ASC NULLS FIRST], false, 0 -(34) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] +(33) ReusedExchange [Reuses operator id: 23] +Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] -(35) HashAggregate [codegen id : 20] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] -Keys [5]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31] -Functions [1]: [sum(UnscaledValue(cs_sales_price#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#20] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#20,17,2) AS sum_sales#21] +(34) HashAggregate [codegen id : 19] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] +Keys [5]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30] +Functions [1]: [sum(UnscaledValue(cs_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#32))#20] +Results [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#32))#20,17,2) AS sum_sales#21] -(36) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#34] +(35) Exchange +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21] +Arguments: hashpartitioning(i_category#26, i_brand#27, cc_name#28, 5), ENSURE_REQUIREMENTS, [id=#33] -(37) Sort [codegen id : 21] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 20] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21] +Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0 -(38) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +(37) Window +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21] +Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#21 AS sum_sales#36, rn#35] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35] +(38) Project [codegen id : 21] +Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#21 AS sum_sales#35, rn#34] +Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21, rn#34] -(40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#35 + 1), 5), ENSURE_REQUIREMENTS, [id=#37] +(39) Sort [codegen id : 21] +Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34] +Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, (rn#34 + 1) ASC NULLS FIRST], false, 0 -(41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#35 + 1) ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 24] +(40) SortMergeJoin [codegen id : 22] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#35 + 1)] +Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#34 + 1)] Join condition: None -(43) Project [codegen id : 24] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] - -(44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] +(41) Project [codegen id : 22] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34] -(45) Sort [codegen id : 33] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 +(42) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21] -(46) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] -Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +(43) Sort [codegen id : 31] +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21] +Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST], false, 0 -(47) Project [codegen id : 34] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#21 AS sum_sales#44, rn#43] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21, rn#43] +(44) Window +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21] +Arguments: [rank(d_year#39, d_moy#40) windowspecdefinition(i_category#36, i_brand#37, cc_name#38, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#36, i_brand#37, cc_name#38], [d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST] -(48) Exchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] -Arguments: hashpartitioning(i_category#38, i_brand#39, cc_name#40, (rn#43 - 1), 5), ENSURE_REQUIREMENTS, [id=#45] +(45) Project [codegen id : 32] +Output [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#21 AS sum_sales#42, rn#41] +Input [7]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21, rn#41] -(49) Sort [codegen id : 35] -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, (rn#43 - 1) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 32] +Input [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] +Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, (rn#41 - 1) ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 36] +(47) SortMergeJoin [codegen id : 33] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#43 - 1)] +Right keys [4]: [i_category#36, i_brand#37, cc_name#38, (rn#41 - 1)] Join condition: None -(51) Project [codegen id : 36] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#46, sum_sales#44 AS nsum#47] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] +(48) Project [codegen id : 33] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#43, sum_sales#42 AS nsum#44] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] -(52) TakeOrderedAndProject -Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] +(49) TakeOrderedAndProject +Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (56) -+- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet default.date_dim (53) +BroadcastExchange (53) ++- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(53) Scan parquet default.date_dim +(50) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -(55) Filter [codegen id : 1] +(52) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6)) -(56) BroadcastExchange +(53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt index 3bf10f82e6a88..b488806fe9a07 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt @@ -1,104 +1,95 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (36) + WholeStageCodegen (33) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (22) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,rn] - InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #1 - WholeStageCodegen (11) - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (10) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (9) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,cc_name] #2 - WholeStageCodegen (8) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3 - WholeStageCodegen (7) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - SortMergeJoin [cs_item_sk,i_item_sk] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (10) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (9) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (8) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk] InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Exchange [cs_item_sk] #3 + WholeStageCodegen (3) + Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [cc_call_center_sk,cc_name] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [cc_call_center_sk,cc_name] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_name] + Scan parquet default.call_center [cc_call_center_sk,cc_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk,i_category,i_brand] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_category] + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk,i_category,i_brand] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (21) Sort [i_category,i_brand,cc_name,rn] - InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #8 - WholeStageCodegen (22) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (21) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,cc_name] #9 - WholeStageCodegen (20) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (20) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (19) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - WholeStageCodegen (35) + WholeStageCodegen (32) Sort [i_category,i_brand,cc_name,rn] - InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #10 - WholeStageCodegen (34) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (33) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #9 + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (31) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 4566f30b27d04..51b2f051403e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -1,56 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- * HashAggregate (24) - : : +- Exchange (23) - : : +- * HashAggregate (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.store (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (41) - : +- Exchange (40) - : +- * Project (39) - : +- Window (38) - : +- * Sort (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * Sort (49) - +- Exchange (48) - +- * Project (47) - +- Window (46) - +- * Sort (45) - +- ReusedExchange (44) +TakeOrderedAndProject (49) ++- * Project (48) + +- * SortMergeJoin Inner (47) + :- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (32) + : : +- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Sort (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.store (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (39) + : +- * Project (38) + : +- Window (37) + : +- * Sort (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- ReusedExchange (33) + +- * Sort (46) + +- * Project (45) + +- Window (44) + +- * Sort (43) + +- ReusedExchange (42) (1) Scan parquet default.store_sales @@ -68,7 +65,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) -(4) ReusedExchange [Reuses operator id: 56] +(4) ReusedExchange [Reuses operator id: 53] Output [3]: [d_date_sk#6, d_year#7, d_moy#8] (5) BroadcastHashJoin [codegen id : 3] @@ -192,118 +189,106 @@ Condition : ((isnotnull(avg_monthly_sales#26) AND (avg_monthly_sales#26 > 0.0000 Output [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25] Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, _w0#23, rn#25, avg_monthly_sales#26] -(32) Exchange -Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25] -Arguments: hashpartitioning(i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25, 5), ENSURE_REQUIREMENTS, [id=#27] - -(33) Sort [codegen id : 12] +(32) Sort [codegen id : 11] Input [9]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25] Arguments: [i_category#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST, s_company_name#11 ASC NULLS FIRST, rn#25 ASC NULLS FIRST], false, 0 -(34) ReusedExchange [Reuses operator id: 23] -Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] +(33) ReusedExchange [Reuses operator id: 23] +Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] -(35) HashAggregate [codegen id : 20] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] -Keys [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(ss_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#21] -Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#21,17,2) AS sum_sales#22] +(34) HashAggregate [codegen id : 19] +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] +Keys [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32] +Functions [1]: [sum(UnscaledValue(ss_sales_price#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#34))#21] +Results [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#34))#21,17,2) AS sum_sales#22] -(36) Exchange -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(35) Exchange +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22] +Arguments: hashpartitioning(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, 5), ENSURE_REQUIREMENTS, [id=#35] -(37) Sort [codegen id : 21] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 20] +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST], false, 0 -(38) Window -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +(37) Window +Input [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#27, i_brand#28, s_store_name#29, s_company_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#22 AS sum_sales#38, rn#37] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22, rn#37] +(38) Project [codegen id : 21] +Output [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#22 AS sum_sales#37, rn#36] +Input [8]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum_sales#22, rn#36] -(40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] +(39) Sort [codegen id : 21] +Input [6]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST, s_company_name#30 ASC NULLS FIRST, (rn#36 + 1) ASC NULLS FIRST], false, 0 -(41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 24] +(40) SortMergeJoin [codegen id : 22] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1)] +Right keys [5]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, (rn#36 + 1)] Join condition: None -(43) Project [codegen id : 24] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] - -(44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] +(41) Project [codegen id : 22] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#27, i_brand#28, s_store_name#29, s_company_name#30, sum_sales#37, rn#36] -(45) Sort [codegen id : 33] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +(42) ReusedExchange [Reuses operator id: 35] +Output [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22] -(46) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] +(43) Sort [codegen id : 31] +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 -(47) Project [codegen id : 34] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#22 AS sum_sales#47, rn#46] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22, rn#46] +(44) Window +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, s_store_name#40, s_company_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] -(48) Exchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] +(45) Project [codegen id : 32] +Output [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#22 AS sum_sales#45, rn#44] +Input [8]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#22, rn#44] -(49) Sort [codegen id : 35] -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 32] +Input [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, (rn#44 - 1) ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 36] +(47) SortMergeJoin [codegen id : 33] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1)] +Right keys [5]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, (rn#44 - 1)] Join condition: None -(51) Project [codegen id : 36] -Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#38 AS psum#49, sum_sales#47 AS nsum#50] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] +(48) Project [codegen id : 33] +Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#46, sum_sales#45 AS nsum#47] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] -(52) TakeOrderedAndProject -Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] +(49) TakeOrderedAndProject +Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#46, nsum#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (56) -+- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet default.date_dim (53) +BroadcastExchange (53) ++- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(53) Scan parquet default.date_dim +(50) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -(55) Filter [codegen id : 1] +(52) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6)) -(56) BroadcastExchange +(53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt index 5f64a22717270..65bcf10a8518b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt @@ -1,104 +1,95 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (36) + WholeStageCodegen (33) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (22) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,rn] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #1 - WholeStageCodegen (11) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (10) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (9) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #2 - WholeStageCodegen (8) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3 - WholeStageCodegen (7) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (10) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (9) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (8) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk,s_store_name,s_company_name] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk,s_store_name,s_company_name] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_company_name] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk,i_category,i_brand] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_category] + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk,i_category,i_brand] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (21) Sort [i_category,i_brand,s_store_name,s_company_name,rn] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #8 - WholeStageCodegen (22) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (21) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #9 - WholeStageCodegen (20) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (20) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (19) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - WholeStageCodegen (35) + WholeStageCodegen (32) Sort [i_category,i_brand,s_store_name,s_company_name,rn] - InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #10 - WholeStageCodegen (34) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (33) - Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #9 + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (31) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index 64111eef627d2..e3d76bfea8c2c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -1,74 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Filter (69) - +- * HashAggregate (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin Inner (65) - :- Window (60) - : +- * Sort (59) - : +- Exchange (58) - : +- * Project (57) - : +- * Filter (56) - : +- * SortMergeJoin FullOuter (55) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- Window (18) - : : +- * Sort (17) - : : +- ReusedExchange (16) - : +- * Sort (54) - : +- Exchange (53) - : +- * HashAggregate (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * SortMergeJoin Inner (49) - : :- * Sort (42) - : : +- Exchange (41) - : : +- * Project (40) - : : +- Window (39) - : : +- * Sort (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- Exchange (35) - : : +- * HashAggregate (34) - : : +- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet default.store_sales (28) - : : +- ReusedExchange (31) - : +- * Sort (48) - : +- Exchange (47) - : +- * Project (46) - : +- Window (45) - : +- * Sort (44) - : +- ReusedExchange (43) - +- * Project (64) - +- Window (63) - +- * Sort (62) - +- ReusedExchange (61) +TakeOrderedAndProject (68) ++- * Filter (67) + +- * HashAggregate (66) + +- * HashAggregate (65) + +- * Project (64) + +- * SortMergeJoin Inner (63) + :- Window (58) + : +- * Sort (57) + : +- Exchange (56) + : +- * Project (55) + : +- * Filter (54) + : +- * SortMergeJoin FullOuter (53) + : :- * Sort (26) + : : +- * HashAggregate (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * SortMergeJoin Inner (22) + : : :- * Sort (15) + : : : +- Exchange (14) + : : : +- * Project (13) + : : : +- Window (12) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- Window (18) + : : +- * Sort (17) + : : +- ReusedExchange (16) + : +- * Sort (52) + : +- * HashAggregate (51) + : +- * HashAggregate (50) + : +- * Project (49) + : +- * SortMergeJoin Inner (48) + : :- * Sort (41) + : : +- Exchange (40) + : : +- * Project (39) + : : +- Window (38) + : : +- * Sort (37) + : : +- Exchange (36) + : : +- * HashAggregate (35) + : : +- Exchange (34) + : : +- * HashAggregate (33) + : : +- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Filter (29) + : : : +- * ColumnarToRow (28) + : : : +- Scan parquet default.store_sales (27) + : : +- ReusedExchange (30) + : +- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- Window (44) + : +- * Sort (43) + : +- ReusedExchange (42) + +- * Project (62) + +- Window (61) + +- * Sort (60) + +- ReusedExchange (59) (1) Scan parquet default.web_sales @@ -86,7 +84,7 @@ Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(4) ReusedExchange [Reuses operator id: 75] +(4) ReusedExchange [Reuses operator id: 73] Output [2]: [d_date_sk#5, d_date#6] (5) BroadcastHashJoin [codegen id : 2] @@ -186,240 +184,232 @@ Functions [1]: [sum(sumws#20)] Aggregate Attributes [1]: [sum(sumws#20)#26] Results [3]: [item_sk#11, d_date#6, sum(sumws#20)#26 AS cume_sales#27] -(26) Exchange -Input [3]: [item_sk#11, d_date#6, cume_sales#27] -Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#28] - -(27) Sort [codegen id : 14] +(26) Sort [codegen id : 13] Input [3]: [item_sk#11, d_date#6, cume_sales#27] Arguments: [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(28) Scan parquet default.store_sales -Output [3]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31] +(27) Scan parquet default.store_sales +Output [3]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31] +(28) ColumnarToRow [codegen id : 15] +Input [3]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30] -(30) Filter [codegen id : 16] -Input [3]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31] -Condition : isnotnull(ss_item_sk#29) +(29) Filter [codegen id : 15] +Input [3]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#28) -(31) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#32, d_date#33] +(30) ReusedExchange [Reuses operator id: 73] +Output [2]: [d_date_sk#31, d_date#32] -(32) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#31] -Right keys [1]: [d_date_sk#32] +(31) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#31] Join condition: None -(33) Project [codegen id : 16] -Output [3]: [ss_item_sk#29, ss_sales_price#30, d_date#33] -Input [5]: [ss_item_sk#29, ss_sales_price#30, ss_sold_date_sk#31, d_date_sk#32, d_date#33] - -(34) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#29, ss_sales_price#30, d_date#33] -Keys [2]: [ss_item_sk#29, d_date#33] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#30))] -Aggregate Attributes [1]: [sum#34] -Results [3]: [ss_item_sk#29, d_date#33, sum#35] - -(35) Exchange -Input [3]: [ss_item_sk#29, d_date#33, sum#35] -Arguments: hashpartitioning(ss_item_sk#29, d_date#33, 5), ENSURE_REQUIREMENTS, [id=#36] - -(36) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#29, d_date#33, sum#35] -Keys [2]: [ss_item_sk#29, d_date#33] -Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#37] -Results [4]: [ss_item_sk#29 AS item_sk#38, d_date#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#37,17,2) AS sumss#39, ss_item_sk#29] - -(37) Exchange -Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29] -Arguments: hashpartitioning(ss_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#40] - -(38) Sort [codegen id : 18] -Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29] -Arguments: [ss_item_sk#29 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST], false, 0 - -(39) Window -Input [4]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29] -Arguments: [row_number() windowspecdefinition(ss_item_sk#29, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#41], [ss_item_sk#29], [d_date#33 ASC NULLS FIRST] - -(40) Project [codegen id : 19] -Output [4]: [item_sk#38, d_date#33, sumss#39, rk#41] -Input [5]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29, rk#41] - -(41) Exchange -Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41] -Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, [id=#42] - -(42) Sort [codegen id : 20] -Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41] -Arguments: [item_sk#38 ASC NULLS FIRST], false, 0 - -(43) ReusedExchange [Reuses operator id: 37] -Output [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44] - -(44) Sort [codegen id : 24] -Input [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44] -Arguments: [ss_item_sk#44 ASC NULLS FIRST, d_date#43 ASC NULLS FIRST], false, 0 - -(45) Window -Input [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44] -Arguments: [row_number() windowspecdefinition(ss_item_sk#44, d_date#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#45], [ss_item_sk#44], [d_date#43 ASC NULLS FIRST] - -(46) Project [codegen id : 25] -Output [3]: [item_sk#38 AS item_sk#46, sumss#39 AS sumss#47, rk#45] -Input [5]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44, rk#45] - -(47) Exchange -Input [3]: [item_sk#46, sumss#47, rk#45] -Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#48] - -(48) Sort [codegen id : 26] -Input [3]: [item_sk#46, sumss#47, rk#45] -Arguments: [item_sk#46 ASC NULLS FIRST], false, 0 - -(49) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#38] -Right keys [1]: [item_sk#46] -Join condition: (rk#41 >= rk#45) - -(50) Project [codegen id : 27] -Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#47] -Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#46, sumss#47, rk#45] - -(51) HashAggregate [codegen id : 27] -Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#47] -Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [partial_sum(sumss#47)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [5]: [item_sk#38, d_date#33, sumss#39, sum#51, isEmpty#52] - -(52) HashAggregate [codegen id : 27] -Input [5]: [item_sk#38, d_date#33, sumss#39, sum#51, isEmpty#52] -Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [sum(sumss#47)] -Aggregate Attributes [1]: [sum(sumss#47)#53] -Results [3]: [item_sk#38, d_date#33, sum(sumss#47)#53 AS cume_sales#54] - -(53) Exchange -Input [3]: [item_sk#38, d_date#33, cume_sales#54] -Arguments: hashpartitioning(item_sk#38, d_date#33, 5), ENSURE_REQUIREMENTS, [id=#55] - -(54) Sort [codegen id : 28] -Input [3]: [item_sk#38, d_date#33, cume_sales#54] -Arguments: [item_sk#38 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST], false, 0 - -(55) SortMergeJoin [codegen id : 29] +(32) Project [codegen id : 15] +Output [3]: [ss_item_sk#28, ss_sales_price#29, d_date#32] +Input [5]: [ss_item_sk#28, ss_sales_price#29, ss_sold_date_sk#30, d_date_sk#31, d_date#32] + +(33) HashAggregate [codegen id : 15] +Input [3]: [ss_item_sk#28, ss_sales_price#29, d_date#32] +Keys [2]: [ss_item_sk#28, d_date#32] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum#33] +Results [3]: [ss_item_sk#28, d_date#32, sum#34] + +(34) Exchange +Input [3]: [ss_item_sk#28, d_date#32, sum#34] +Arguments: hashpartitioning(ss_item_sk#28, d_date#32, 5), ENSURE_REQUIREMENTS, [id=#35] + +(35) HashAggregate [codegen id : 16] +Input [3]: [ss_item_sk#28, d_date#32, sum#34] +Keys [2]: [ss_item_sk#28, d_date#32] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#36] +Results [4]: [ss_item_sk#28 AS item_sk#37, d_date#32, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#36,17,2) AS sumss#38, ss_item_sk#28] + +(36) Exchange +Input [4]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28] +Arguments: hashpartitioning(ss_item_sk#28, 5), ENSURE_REQUIREMENTS, [id=#39] + +(37) Sort [codegen id : 17] +Input [4]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28] +Arguments: [ss_item_sk#28 ASC NULLS FIRST, d_date#32 ASC NULLS FIRST], false, 0 + +(38) Window +Input [4]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28] +Arguments: [row_number() windowspecdefinition(ss_item_sk#28, d_date#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#40], [ss_item_sk#28], [d_date#32 ASC NULLS FIRST] + +(39) Project [codegen id : 18] +Output [4]: [item_sk#37, d_date#32, sumss#38, rk#40] +Input [5]: [item_sk#37, d_date#32, sumss#38, ss_item_sk#28, rk#40] + +(40) Exchange +Input [4]: [item_sk#37, d_date#32, sumss#38, rk#40] +Arguments: hashpartitioning(item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] + +(41) Sort [codegen id : 19] +Input [4]: [item_sk#37, d_date#32, sumss#38, rk#40] +Arguments: [item_sk#37 ASC NULLS FIRST], false, 0 + +(42) ReusedExchange [Reuses operator id: 36] +Output [4]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43] + +(43) Sort [codegen id : 23] +Input [4]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43] +Arguments: [ss_item_sk#43 ASC NULLS FIRST, d_date#42 ASC NULLS FIRST], false, 0 + +(44) Window +Input [4]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43] +Arguments: [row_number() windowspecdefinition(ss_item_sk#43, d_date#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [ss_item_sk#43], [d_date#42 ASC NULLS FIRST] + +(45) Project [codegen id : 24] +Output [3]: [item_sk#37 AS item_sk#45, sumss#38 AS sumss#46, rk#44] +Input [5]: [item_sk#37, d_date#42, sumss#38, ss_item_sk#43, rk#44] + +(46) Exchange +Input [3]: [item_sk#45, sumss#46, rk#44] +Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#47] + +(47) Sort [codegen id : 25] +Input [3]: [item_sk#45, sumss#46, rk#44] +Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 26] +Left keys [1]: [item_sk#37] +Right keys [1]: [item_sk#45] +Join condition: (rk#40 >= rk#44) + +(49) Project [codegen id : 26] +Output [4]: [item_sk#37, d_date#32, sumss#38, sumss#46] +Input [7]: [item_sk#37, d_date#32, sumss#38, rk#40, item_sk#45, sumss#46, rk#44] + +(50) HashAggregate [codegen id : 26] +Input [4]: [item_sk#37, d_date#32, sumss#38, sumss#46] +Keys [3]: [item_sk#37, d_date#32, sumss#38] +Functions [1]: [partial_sum(sumss#46)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [5]: [item_sk#37, d_date#32, sumss#38, sum#50, isEmpty#51] + +(51) HashAggregate [codegen id : 26] +Input [5]: [item_sk#37, d_date#32, sumss#38, sum#50, isEmpty#51] +Keys [3]: [item_sk#37, d_date#32, sumss#38] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#52] +Results [3]: [item_sk#37, d_date#32, sum(sumss#46)#52 AS cume_sales#53] + +(52) Sort [codegen id : 26] +Input [3]: [item_sk#37, d_date#32, cume_sales#53] +Arguments: [item_sk#37 ASC NULLS FIRST, d_date#32 ASC NULLS FIRST], false, 0 + +(53) SortMergeJoin [codegen id : 27] Left keys [2]: [item_sk#11, d_date#6] -Right keys [2]: [item_sk#38, d_date#33] +Right keys [2]: [item_sk#37, d_date#32] Join condition: None -(56) Filter [codegen id : 29] -Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#38, d_date#33, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#38 END) +(54) Filter [codegen id : 27] +Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#37, d_date#32, cume_sales#53] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#37 END) -(57) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#38 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#33 END AS d_date#57, cume_sales#27 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#38, d_date#33, cume_sales#54] +(55) Project [codegen id : 27] +Output [4]: [CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#37 END AS item_sk#54, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#32 END AS d_date#55, cume_sales#27 AS web_sales#56, cume_sales#53 AS store_sales#57] +Input [6]: [item_sk#11, d_date#6, cume_sales#27, item_sk#37, d_date#32, cume_sales#53] -(58) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] +(56) Exchange +Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Arguments: hashpartitioning(item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(59) Sort [codegen id : 30] -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 28] +Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Arguments: [item_sk#54 ASC NULLS FIRST, d_date#55 ASC NULLS FIRST], false, 0 -(60) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] +(58) Window +Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Arguments: [row_number() windowspecdefinition(item_sk#54, d_date#55 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#59], [item_sk#54], [d_date#55 ASC NULLS FIRST] -(61) ReusedExchange [Reuses operator id: 58] -Output [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +(59) ReusedExchange [Reuses operator id: 56] +Output [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] -(62) Sort [codegen id : 60] -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 +(60) Sort [codegen id : 56] +Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Arguments: [item_sk#54 ASC NULLS FIRST, d_date#55 ASC NULLS FIRST], false, 0 -(63) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#62], [item_sk#56], [d_date#57 ASC NULLS FIRST] +(61) Window +Input [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Arguments: [row_number() windowspecdefinition(item_sk#54, d_date#55 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#60], [item_sk#54], [d_date#55 ASC NULLS FIRST] -(64) Project [codegen id : 61] -Output [4]: [item_sk#56 AS item_sk#63, web_sales#58 AS web_sales#64, store_sales#59 AS store_sales#65, rk#62] -Input [5]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#62] +(62) Project [codegen id : 57] +Output [4]: [item_sk#54 AS item_sk#61, web_sales#56 AS web_sales#62, store_sales#57 AS store_sales#63, rk#60] +Input [5]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, rk#60] -(65) SortMergeJoin [codegen id : 62] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#63] -Join condition: (rk#61 >= rk#62) +(63) SortMergeJoin [codegen id : 58] +Left keys [1]: [item_sk#54] +Right keys [1]: [item_sk#61] +Join condition: (rk#59 >= rk#60) -(66) Project [codegen id : 62] -Output [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65] -Input [9]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#61, item_sk#63, web_sales#64, store_sales#65, rk#62] +(64) Project [codegen id : 58] +Output [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_sales#62, store_sales#63] +Input [9]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, rk#59, item_sk#61, web_sales#62, store_sales#63, rk#60] -(67) HashAggregate [codegen id : 62] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65] -Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Functions [2]: [partial_max(web_sales#64), partial_max(store_sales#65)] -Aggregate Attributes [2]: [max#66, max#67] -Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#68, max#69] +(65) HashAggregate [codegen id : 58] +Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_sales#62, store_sales#63] +Keys [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Functions [2]: [partial_max(web_sales#62), partial_max(store_sales#63)] +Aggregate Attributes [2]: [max#64, max#65] +Results [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, max#66, max#67] -(68) HashAggregate [codegen id : 62] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#68, max#69] -Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Functions [2]: [max(web_sales#64), max(store_sales#65)] -Aggregate Attributes [2]: [max(web_sales#64)#70, max(store_sales#65)#71] -Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max(web_sales#64)#70 AS web_cumulative#72, max(store_sales#65)#71 AS store_cumulative#73] +(66) HashAggregate [codegen id : 58] +Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, max#66, max#67] +Keys [4]: [item_sk#54, d_date#55, web_sales#56, store_sales#57] +Functions [2]: [max(web_sales#62), max(store_sales#63)] +Aggregate Attributes [2]: [max(web_sales#62)#68, max(store_sales#63)#69] +Results [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, max(web_sales#62)#68 AS web_cumulative#70, max(store_sales#63)#69 AS store_cumulative#71] -(69) Filter [codegen id : 62] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] -Condition : ((isnotnull(web_cumulative#72) AND isnotnull(store_cumulative#73)) AND (web_cumulative#72 > store_cumulative#73)) +(67) Filter [codegen id : 58] +Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_cumulative#70, store_cumulative#71] +Condition : ((isnotnull(web_cumulative#70) AND isnotnull(store_cumulative#71)) AND (web_cumulative#70 > store_cumulative#71)) -(70) TakeOrderedAndProject -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] -Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] +(68) TakeOrderedAndProject +Input [6]: [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_cumulative#70, store_cumulative#71] +Arguments: 100, [item_sk#54 ASC NULLS FIRST, d_date#55 ASC NULLS FIRST], [item_sk#54, d_date#55, web_sales#56, store_sales#57, web_cumulative#70, store_cumulative#71] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (75) -+- * Project (74) - +- * Filter (73) - +- * ColumnarToRow (72) - +- Scan parquet default.date_dim (71) +BroadcastExchange (73) ++- * Project (72) + +- * Filter (71) + +- * ColumnarToRow (70) + +- Scan parquet default.date_dim (69) -(71) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#74] +(69) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#72] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74] +(70) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72] -(73) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1212)) AND (d_month_seq#74 <= 1223)) AND isnotnull(d_date_sk#5)) +(71) Filter [codegen id : 1] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72] +Condition : (((isnotnull(d_month_seq#72) AND (d_month_seq#72 >= 1212)) AND (d_month_seq#72 <= 1223)) AND isnotnull(d_date_sk#5)) -(74) Project [codegen id : 1] +(72) Project [codegen id : 1] Output [2]: [d_date_sk#5, d_date#6] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72] -(75) BroadcastExchange +(73) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] -Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt index 1a89b7c72a169..b1d245a9ffc43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (62) + WholeStageCodegen (58) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,129 +7,123 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SortMergeJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (30) + WholeStageCodegen (28) Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 - WholeStageCodegen (29) + WholeStageCodegen (27) Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] Filter [item_sk,item_sk] SortMergeJoin [item_sk,d_date,item_sk,d_date] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (13) Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (13) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - SortMergeJoin [item_sk,item_sk,rk,rk] + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + SortMergeJoin [item_sk,item_sk,rk,rk] + InputAdapter + WholeStageCodegen (6) + Sort [item_sk] InputAdapter - WholeStageCodegen (6) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #3 - WholeStageCodegen (5) - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - Sort [ws_item_sk,d_date] - InputAdapter - Exchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - InputAdapter - Exchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date] - Filter [d_month_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Exchange [item_sk] #2 + WholeStageCodegen (5) + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + Sort [ws_item_sk,d_date] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + Exchange [ws_item_sk,d_date] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [item_sk] InputAdapter - WholeStageCodegen (12) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #7 - WholeStageCodegen (11) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (10) - Sort [ws_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + Exchange [item_sk] #6 + WholeStageCodegen (11) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (10) + Sort [ws_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #3 InputAdapter - WholeStageCodegen (28) + WholeStageCodegen (26) Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #8 - WholeStageCodegen (27) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - SortMergeJoin [item_sk,item_sk,rk,rk] + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + SortMergeJoin [item_sk,item_sk,rk,rk] + InputAdapter + WholeStageCodegen (19) + Sort [item_sk] InputAdapter - WholeStageCodegen (20) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #9 - WholeStageCodegen (19) - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (18) - Sort [ss_item_sk,d_date] - InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - InputAdapter - Exchange [ss_item_sk,d_date] #11 - WholeStageCodegen (16) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Exchange [item_sk] #7 + WholeStageCodegen (18) + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (17) + Sort [ss_item_sk,d_date] + InputAdapter + Exchange [ss_item_sk] #8 + WholeStageCodegen (16) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + Exchange [ss_item_sk,d_date] #9 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (25) + Sort [item_sk] InputAdapter - WholeStageCodegen (26) - Sort [item_sk] - InputAdapter - Exchange [item_sk] #12 - WholeStageCodegen (25) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (24) - Sort [ss_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + Exchange [item_sk] #10 + WholeStageCodegen (24) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (23) + Sort [ss_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #8 InputAdapter - WholeStageCodegen (61) + WholeStageCodegen (57) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (60) + WholeStageCodegen (56) Sort [item_sk,d_date] InputAdapter ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index d214b321a4791..aa9b899a9308c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -1,56 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * Project (51) - +- * SortMergeJoin Inner (50) - :- * Project (43) - : +- * SortMergeJoin Inner (42) - : :- * Sort (33) - : : +- Exchange (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Sort (26) - : : +- Exchange (25) - : : +- * HashAggregate (24) - : : +- Exchange (23) - : : +- * HashAggregate (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.catalog_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.call_center (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (41) - : +- Exchange (40) - : +- * Project (39) - : +- Window (38) - : +- * Sort (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * Sort (49) - +- Exchange (48) - +- * Project (47) - +- Window (46) - +- * Sort (45) - +- ReusedExchange (44) +TakeOrderedAndProject (49) ++- * Project (48) + +- * SortMergeJoin Inner (47) + :- * Project (41) + : +- * SortMergeJoin Inner (40) + : :- * Sort (32) + : : +- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Sort (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- Exchange (23) + : : +- * HashAggregate (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.catalog_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.call_center (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (39) + : +- * Project (38) + : +- Window (37) + : +- * Sort (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- ReusedExchange (33) + +- * Sort (46) + +- * Project (45) + +- Window (44) + +- * Sort (43) + +- ReusedExchange (42) (1) Scan parquet default.catalog_sales @@ -68,7 +65,7 @@ Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk Input [4]: [cs_call_center_sk#1, cs_item_sk#2, cs_sales_price#3, cs_sold_date_sk#4] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_call_center_sk#1)) -(4) ReusedExchange [Reuses operator id: 56] +(4) ReusedExchange [Reuses operator id: 53] Output [3]: [d_date_sk#6, d_year#7, d_moy#8] (5) BroadcastHashJoin [codegen id : 3] @@ -192,118 +189,106 @@ Condition : ((isnotnull(avg_monthly_sales#25) AND (avg_monthly_sales#25 > 0.0000 Output [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24] Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, _w0#22, rn#24, avg_monthly_sales#25] -(32) Exchange -Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24] -Arguments: hashpartitioning(i_category#15, i_brand#14, cc_name#10, rn#24, 5), ENSURE_REQUIREMENTS, [id=#26] - -(33) Sort [codegen id : 12] +(32) Sort [codegen id : 11] Input [8]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24] Arguments: [i_category#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST, rn#24 ASC NULLS FIRST], false, 0 -(34) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] +(33) ReusedExchange [Reuses operator id: 23] +Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] -(35) HashAggregate [codegen id : 20] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] -Keys [5]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31] -Functions [1]: [sum(UnscaledValue(cs_sales_price#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#20] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#20,17,2) AS sum_sales#21] +(34) HashAggregate [codegen id : 19] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] +Keys [5]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30] +Functions [1]: [sum(UnscaledValue(cs_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#32))#20] +Results [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, MakeDecimal(sum(UnscaledValue(cs_sales_price#32))#20,17,2) AS sum_sales#21] -(36) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#34] +(35) Exchange +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21] +Arguments: hashpartitioning(i_category#26, i_brand#27, cc_name#28, 5), ENSURE_REQUIREMENTS, [id=#33] -(37) Sort [codegen id : 21] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 20] +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21] +Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST], false, 0 -(38) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +(37) Window +Input [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21] +Arguments: [rank(d_year#29, d_moy#30) windowspecdefinition(i_category#26, i_brand#27, cc_name#28, d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#26, i_brand#27, cc_name#28], [d_year#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST] -(39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#21 AS sum_sales#36, rn#35] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35] +(38) Project [codegen id : 21] +Output [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#21 AS sum_sales#35, rn#34] +Input [7]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum_sales#21, rn#34] -(40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#35 + 1), 5), ENSURE_REQUIREMENTS, [id=#37] +(39) Sort [codegen id : 21] +Input [5]: [i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34] +Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, cc_name#28 ASC NULLS FIRST, (rn#34 + 1) ASC NULLS FIRST], false, 0 -(41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#35 + 1) ASC NULLS FIRST], false, 0 - -(42) SortMergeJoin [codegen id : 24] +(40) SortMergeJoin [codegen id : 22] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#35 + 1)] +Right keys [4]: [i_category#26, i_brand#27, cc_name#28, (rn#34 + 1)] Join condition: None -(43) Project [codegen id : 24] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] - -(44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] +(41) Project [codegen id : 22] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, cc_name#28, sum_sales#35, rn#34] -(45) Sort [codegen id : 33] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 +(42) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21] -(46) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] -Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +(43) Sort [codegen id : 31] +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21] +Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST], false, 0 -(47) Project [codegen id : 34] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#21 AS sum_sales#44, rn#43] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21, rn#43] +(44) Window +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21] +Arguments: [rank(d_year#39, d_moy#40) windowspecdefinition(i_category#36, i_brand#37, cc_name#38, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#36, i_brand#37, cc_name#38], [d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST] -(48) Exchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] -Arguments: hashpartitioning(i_category#38, i_brand#39, cc_name#40, (rn#43 - 1), 5), ENSURE_REQUIREMENTS, [id=#45] +(45) Project [codegen id : 32] +Output [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#21 AS sum_sales#42, rn#41] +Input [7]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#21, rn#41] -(49) Sort [codegen id : 35] -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] -Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, (rn#43 - 1) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 32] +Input [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] +Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, (rn#41 - 1) ASC NULLS FIRST], false, 0 -(50) SortMergeJoin [codegen id : 36] +(47) SortMergeJoin [codegen id : 33] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#43 - 1)] +Right keys [4]: [i_category#36, i_brand#37, cc_name#38, (rn#41 - 1)] Join condition: None -(51) Project [codegen id : 36] -Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#46, sum_sales#44 AS nsum#47] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] +(48) Project [codegen id : 33] +Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#43, sum_sales#42 AS nsum#44] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] -(52) TakeOrderedAndProject -Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] +(49) TakeOrderedAndProject +Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#43, nsum#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (56) -+- * Filter (55) - +- * ColumnarToRow (54) - +- Scan parquet default.date_dim (53) +BroadcastExchange (53) ++- * Filter (52) + +- * ColumnarToRow (51) + +- Scan parquet default.date_dim (50) -(53) Scan parquet default.date_dim +(50) Scan parquet default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 1] +(51) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -(55) Filter [codegen id : 1] +(52) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ((d_year#7 = 2000) AND (d_moy#8 = 1))) AND isnotnull(d_date_sk#6)) -(56) BroadcastExchange +(53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt index b464f558bbc1a..4389f6035a41b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt @@ -1,104 +1,95 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (36) + WholeStageCodegen (33) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (22) Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,rn] - InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #1 - WholeStageCodegen (11) - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (10) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (9) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,cc_name] #2 - WholeStageCodegen (8) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - InputAdapter - Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3 - WholeStageCodegen (7) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - SortMergeJoin [cs_item_sk,i_item_sk] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (10) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (9) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (8) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk] InputAdapter - WholeStageCodegen (4) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Exchange [cs_item_sk] #3 + WholeStageCodegen (3) + Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [cc_call_center_sk,cc_name] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [cc_call_center_sk,cc_name] - ColumnarToRow - InputAdapter - Scan parquet default.call_center [cc_call_center_sk,cc_name] + Scan parquet default.call_center [cc_call_center_sk,cc_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk,i_category,i_brand] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand,i_category] + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk,i_category,i_brand] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter - WholeStageCodegen (23) + WholeStageCodegen (21) Sort [i_category,i_brand,cc_name,rn] - InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #8 - WholeStageCodegen (22) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (21) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - Exchange [i_category,i_brand,cc_name] #9 - WholeStageCodegen (20) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (20) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (19) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - WholeStageCodegen (35) + WholeStageCodegen (32) Sort [i_category,i_brand,cc_name,rn] - InputAdapter - Exchange [i_category,i_brand,cc_name,rn] #10 - WholeStageCodegen (34) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (33) - Sort [i_category,i_brand,cc_name,d_year,d_moy] - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #9 + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (31) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index ddaa34ab4e657..cfee2290adff9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,187 +1,185 @@ == Physical Plan == -* Sort (183) -+- Exchange (182) - +- * Project (181) - +- * SortMergeJoin Inner (180) - :- * Sort (111) - : +- Exchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (100) - : : +- * BroadcastHashJoin Inner BuildRight (99) - : : :- * Project (97) - : : : +- * BroadcastHashJoin Inner BuildRight (96) - : : : :- * Project (91) - : : : : +- * BroadcastHashJoin Inner BuildRight (90) - : : : : :- * Project (88) - : : : : : +- * BroadcastHashJoin Inner BuildRight (87) - : : : : : :- * Project (82) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : : :- * Project (79) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : : :- * Project (73) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) - : : : : : : : : :- * Project (67) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : : : : : : : :- * Project (64) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : : :- * Project (58) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : : :- * Project (49) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : : : : : : : : : : :- * Project (43) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : : : : : : : : : : : :- * Project (37) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : : : : : : : : : : : : :- * Project (34) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (33) - : : : : : : : : : : : : : : : : :- * Sort (12) - : : : : : : : : : : : : : : : : : +- Exchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * Project (8) - : : : : : : : : : : : : : : : : : +- * Filter (7) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (5) - : : : : : : : : : : : : : : : : +- * Sort (32) - : : : : : : : : : : : : : : : : +- * Project (31) - : : : : : : : : : : : : : : : : +- * Filter (30) - : : : : : : : : : : : : : : : : +- * HashAggregate (29) - : : : : : : : : : : : : : : : : +- Exchange (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- * Project (26) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (25) - : : : : : : : : : : : : : : : : :- * Sort (18) - : : : : : : : : : : : : : : : : : +- Exchange (17) - : : : : : : : : : : : : : : : : : +- * Project (16) - : : : : : : : : : : : : : : : : : +- * Filter (15) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- * Sort (24) - : : : : : : : : : : : : : : : : +- Exchange (23) - : : : : : : : : : : : : : : : : +- * Project (22) - : : : : : : : : : : : : : : : : +- * Filter (21) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (20) - : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_returns (19) - : : : : : : : : : : : : : : : +- ReusedExchange (35) - : : : : : : : : : : : : : : +- BroadcastExchange (41) - : : : : : : : : : : : : : : +- * Filter (40) - : : : : : : : : : : : : : : +- * ColumnarToRow (39) - : : : : : : : : : : : : : : +- Scan parquet default.store (38) - : : : : : : : : : : : : : +- BroadcastExchange (47) - : : : : : : : : : : : : : +- * Filter (46) - : : : : : : : : : : : : : +- * ColumnarToRow (45) - : : : : : : : : : : : : : +- Scan parquet default.customer (44) - : : : : : : : : : : : : +- BroadcastExchange (53) - : : : : : : : : : : : : +- * Filter (52) - : : : : : : : : : : : : +- * ColumnarToRow (51) - : : : : : : : : : : : : +- Scan parquet default.date_dim (50) - : : : : : : : : : : : +- ReusedExchange (56) - : : : : : : : : : : +- BroadcastExchange (62) - : : : : : : : : : : +- * Filter (61) - : : : : : : : : : : +- * ColumnarToRow (60) - : : : : : : : : : : +- Scan parquet default.customer_demographics (59) - : : : : : : : : : +- ReusedExchange (65) - : : : : : : : : +- BroadcastExchange (71) - : : : : : : : : +- * Filter (70) - : : : : : : : : +- * ColumnarToRow (69) - : : : : : : : : +- Scan parquet default.promotion (68) - : : : : : : : +- BroadcastExchange (77) - : : : : : : : +- * Filter (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- Scan parquet default.household_demographics (74) - : : : : : : +- ReusedExchange (80) - : : : : : +- BroadcastExchange (86) - : : : : : +- * Filter (85) - : : : : : +- * ColumnarToRow (84) - : : : : : +- Scan parquet default.customer_address (83) - : : : : +- ReusedExchange (89) - : : : +- BroadcastExchange (95) - : : : +- * Filter (94) - : : : +- * ColumnarToRow (93) - : : : +- Scan parquet default.income_band (92) - : : +- ReusedExchange (98) - : +- BroadcastExchange (105) - : +- * Project (104) - : +- * Filter (103) - : +- * ColumnarToRow (102) - : +- Scan parquet default.item (101) - +- * Sort (179) - +- Exchange (178) - +- * HashAggregate (177) - +- * HashAggregate (176) - +- * Project (175) - +- * BroadcastHashJoin Inner BuildRight (174) - :- * Project (172) - : +- * BroadcastHashJoin Inner BuildRight (171) - : :- * Project (169) - : : +- * BroadcastHashJoin Inner BuildRight (168) - : : :- * Project (166) - : : : +- * BroadcastHashJoin Inner BuildRight (165) - : : : :- * Project (163) - : : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : : :- * Project (160) - : : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : : :- * Project (157) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : : :- * Project (154) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : : :- * Project (151) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : : :- * Project (148) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : : :- * Project (145) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : : :- * Project (142) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : : :- * Project (133) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : : : : : : : : : : : :- * Project (130) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (129) - : : : : : : : : : : : : : : : :- * Sort (123) - : : : : : : : : : : : : : : : : +- Exchange (122) - : : : : : : : : : : : : : : : : +- * Project (121) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (120) - : : : : : : : : : : : : : : : : :- BroadcastExchange (115) - : : : : : : : : : : : : : : : : : +- * Filter (114) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (113) - : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (112) - : : : : : : : : : : : : : : : : +- * Project (119) - : : : : : : : : : : : : : : : : +- * Filter (118) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (117) - : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (116) - : : : : : : : : : : : : : : : +- * Sort (128) - : : : : : : : : : : : : : : : +- * Project (127) - : : : : : : : : : : : : : : : +- * Filter (126) - : : : : : : : : : : : : : : : +- * HashAggregate (125) - : : : : : : : : : : : : : : : +- ReusedExchange (124) - : : : : : : : : : : : : : : +- ReusedExchange (131) - : : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : : +- ReusedExchange (146) - : : : : : : : : +- ReusedExchange (149) - : : : : : : : +- ReusedExchange (152) - : : : : : : +- ReusedExchange (155) - : : : : : +- ReusedExchange (158) - : : : : +- ReusedExchange (161) - : : : +- ReusedExchange (164) - : : +- ReusedExchange (167) - : +- ReusedExchange (170) - +- ReusedExchange (173) +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- * HashAggregate (109) + : +- * HashAggregate (108) + : +- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * Project (100) + : : +- * BroadcastHashJoin Inner BuildRight (99) + : : :- * Project (97) + : : : +- * BroadcastHashJoin Inner BuildRight (96) + : : : :- * Project (91) + : : : : +- * BroadcastHashJoin Inner BuildRight (90) + : : : : :- * Project (88) + : : : : : +- * BroadcastHashJoin Inner BuildRight (87) + : : : : : :- * Project (82) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) + : : : : : : :- * Project (79) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : : :- * Project (73) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) + : : : : : : : : :- * Project (67) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) + : : : : : : : : : :- * Project (64) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : : : : : : : :- * Project (58) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : : :- * Project (49) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : : : : : : : : : : :- * Project (43) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : : : : : : : : : : :- * Project (37) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : : : : : : : : : : : : :- * Project (34) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (33) + : : : : : : : : : : : : : : : : :- * Sort (12) + : : : : : : : : : : : : : : : : : +- Exchange (11) + : : : : : : : : : : : : : : : : : +- * Project (10) + : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) + : : : : : : : : : : : : : : : : : : +- * Filter (3) + : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (1) + : : : : : : : : : : : : : : : : : +- * Project (8) + : : : : : : : : : : : : : : : : : +- * Filter (7) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (5) + : : : : : : : : : : : : : : : : +- * Sort (32) + : : : : : : : : : : : : : : : : +- * Project (31) + : : : : : : : : : : : : : : : : +- * Filter (30) + : : : : : : : : : : : : : : : : +- * HashAggregate (29) + : : : : : : : : : : : : : : : : +- Exchange (28) + : : : : : : : : : : : : : : : : +- * HashAggregate (27) + : : : : : : : : : : : : : : : : +- * Project (26) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (25) + : : : : : : : : : : : : : : : : :- * Sort (18) + : : : : : : : : : : : : : : : : : +- Exchange (17) + : : : : : : : : : : : : : : : : : +- * Project (16) + : : : : : : : : : : : : : : : : : +- * Filter (15) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : : : : : : : : : : : +- * Sort (24) + : : : : : : : : : : : : : : : : +- Exchange (23) + : : : : : : : : : : : : : : : : +- * Project (22) + : : : : : : : : : : : : : : : : +- * Filter (21) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (20) + : : : : : : : : : : : : : : : : +- Scan parquet default.catalog_returns (19) + : : : : : : : : : : : : : : : +- ReusedExchange (35) + : : : : : : : : : : : : : : +- BroadcastExchange (41) + : : : : : : : : : : : : : : +- * Filter (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- Scan parquet default.store (38) + : : : : : : : : : : : : : +- BroadcastExchange (47) + : : : : : : : : : : : : : +- * Filter (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- Scan parquet default.customer (44) + : : : : : : : : : : : : +- BroadcastExchange (53) + : : : : : : : : : : : : +- * Filter (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- Scan parquet default.date_dim (50) + : : : : : : : : : : : +- ReusedExchange (56) + : : : : : : : : : : +- BroadcastExchange (62) + : : : : : : : : : : +- * Filter (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- Scan parquet default.customer_demographics (59) + : : : : : : : : : +- ReusedExchange (65) + : : : : : : : : +- BroadcastExchange (71) + : : : : : : : : +- * Filter (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- Scan parquet default.promotion (68) + : : : : : : : +- BroadcastExchange (77) + : : : : : : : +- * Filter (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- Scan parquet default.household_demographics (74) + : : : : : : +- ReusedExchange (80) + : : : : : +- BroadcastExchange (86) + : : : : : +- * Filter (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- Scan parquet default.customer_address (83) + : : : : +- ReusedExchange (89) + : : : +- BroadcastExchange (95) + : : : +- * Filter (94) + : : : +- * ColumnarToRow (93) + : : : +- Scan parquet default.income_band (92) + : : +- ReusedExchange (98) + : +- BroadcastExchange (105) + : +- * Project (104) + : +- * Filter (103) + : +- * ColumnarToRow (102) + : +- Scan parquet default.item (101) + +- * Sort (177) + +- * HashAggregate (176) + +- * HashAggregate (175) + +- * Project (174) + +- * BroadcastHashJoin Inner BuildRight (173) + :- * Project (171) + : +- * BroadcastHashJoin Inner BuildRight (170) + : :- * Project (168) + : : +- * BroadcastHashJoin Inner BuildRight (167) + : : :- * Project (165) + : : : +- * BroadcastHashJoin Inner BuildRight (164) + : : : :- * Project (162) + : : : : +- * BroadcastHashJoin Inner BuildRight (161) + : : : : :- * Project (159) + : : : : : +- * BroadcastHashJoin Inner BuildRight (158) + : : : : : :- * Project (156) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (155) + : : : : : : :- * Project (153) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (152) + : : : : : : : :- * Project (150) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (149) + : : : : : : : : :- * Project (147) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (146) + : : : : : : : : : :- * Project (144) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (143) + : : : : : : : : : : :- * Project (141) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (140) + : : : : : : : : : : : :- * Project (138) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (137) + : : : : : : : : : : : : :- * Project (135) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (134) + : : : : : : : : : : : : : :- * Project (132) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) + : : : : : : : : : : : : : : :- * Project (129) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (128) + : : : : : : : : : : : : : : : :- * Sort (122) + : : : : : : : : : : : : : : : : +- Exchange (121) + : : : : : : : : : : : : : : : : +- * Project (120) + : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (119) + : : : : : : : : : : : : : : : : :- BroadcastExchange (114) + : : : : : : : : : : : : : : : : : +- * Filter (113) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (112) + : : : : : : : : : : : : : : : : : +- Scan parquet default.store_sales (111) + : : : : : : : : : : : : : : : : +- * Project (118) + : : : : : : : : : : : : : : : : +- * Filter (117) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (116) + : : : : : : : : : : : : : : : : +- Scan parquet default.store_returns (115) + : : : : : : : : : : : : : : : +- * Sort (127) + : : : : : : : : : : : : : : : +- * Project (126) + : : : : : : : : : : : : : : : +- * Filter (125) + : : : : : : : : : : : : : : : +- * HashAggregate (124) + : : : : : : : : : : : : : : : +- ReusedExchange (123) + : : : : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : : : : +- ReusedExchange (133) + : : : : : : : : : : : : +- ReusedExchange (136) + : : : : : : : : : : : +- ReusedExchange (139) + : : : : : : : : : : +- ReusedExchange (142) + : : : : : : : : : +- ReusedExchange (145) + : : : : : : : : +- ReusedExchange (148) + : : : : : : : +- ReusedExchange (151) + : : : : : : +- ReusedExchange (154) + : : : : : +- ReusedExchange (157) + : : : : +- ReusedExchange (160) + : : : +- ReusedExchange (163) + : : +- ReusedExchange (166) + : +- ReusedExchange (169) + +- ReusedExchange (172) (1) Scan parquet default.store_sales @@ -338,7 +336,7 @@ Join condition: None Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] -(35) ReusedExchange [Reuses operator id: 187] +(35) ReusedExchange [Reuses operator id: 185] Output [2]: [d_date_sk#42, d_year#43] (36) BroadcastHashJoin [codegen id : 25] @@ -671,368 +669,360 @@ Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledV Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#9))#100, sum(UnscaledValue(ss_list_price#10))#101, sum(UnscaledValue(ss_coupon_amt#11))#102] Results [17]: [i_product_name#89 AS product_name#103, i_item_sk#86 AS item_sk#104, s_store_name#45 AS store_name#105, s_zip#46 AS store_zip#106, ca_street_number#73 AS b_street_number#107, ca_street_name#74 AS b_streen_name#108, ca_city#75 AS b_city#109, ca_zip#76 AS b_zip#110, ca_street_number#79 AS c_street_number#111, ca_street_name#80 AS c_street_name#112, ca_city#81 AS c_city#113, ca_zip#82 AS c_zip#114, d_year#43 AS syear#115, count(1)#99 AS cnt#116, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#100,17,2) AS s1#117, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#101,17,2) AS s2#118, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#102,17,2) AS s3#119] -(110) Exchange -Input [17]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119] -Arguments: hashpartitioning(item_sk#104, store_name#105, store_zip#106, 5), ENSURE_REQUIREMENTS, [id=#120] - -(111) Sort [codegen id : 26] +(110) Sort [codegen id : 25] Input [17]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119] Arguments: [item_sk#104 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, store_zip#106 ASC NULLS FIRST], false, 0 -(112) Scan parquet default.store_sales -Output [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] +(111) Scan parquet default.store_sales +Output [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#132), dynamicpruningexpression(ss_sold_date_sk#132 IN dynamicpruning#133)] +PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(113) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] +(112) ColumnarToRow [codegen id : 26] +Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] -(114) Filter [codegen id : 27] -Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -Condition : (((((((isnotnull(ss_item_sk#121) AND isnotnull(ss_ticket_number#128)) AND isnotnull(ss_store_sk#126)) AND isnotnull(ss_customer_sk#122)) AND isnotnull(ss_cdemo_sk#123)) AND isnotnull(ss_promo_sk#127)) AND isnotnull(ss_hdemo_sk#124)) AND isnotnull(ss_addr_sk#125)) +(113) Filter [codegen id : 26] +Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Condition : (((((((isnotnull(ss_item_sk#120) AND isnotnull(ss_ticket_number#127)) AND isnotnull(ss_store_sk#125)) AND isnotnull(ss_customer_sk#121)) AND isnotnull(ss_cdemo_sk#122)) AND isnotnull(ss_promo_sk#126)) AND isnotnull(ss_hdemo_sk#123)) AND isnotnull(ss_addr_sk#124)) -(115) BroadcastExchange -Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#134] +(114) BroadcastExchange +Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#133] -(116) Scan parquet default.store_returns -Output [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137] +(115) Scan parquet default.store_returns +Output [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(117) ColumnarToRow -Input [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137] +(116) ColumnarToRow +Input [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] -(118) Filter -Input [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137] -Condition : (isnotnull(sr_item_sk#135) AND isnotnull(sr_ticket_number#136)) +(117) Filter +Input [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] +Condition : (isnotnull(sr_item_sk#134) AND isnotnull(sr_ticket_number#135)) -(119) Project -Output [2]: [sr_item_sk#135, sr_ticket_number#136] -Input [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137] +(118) Project +Output [2]: [sr_item_sk#134, sr_ticket_number#135] +Input [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] -(120) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [ss_item_sk#121, ss_ticket_number#128] -Right keys [2]: [sr_item_sk#135, sr_ticket_number#136] +(119) BroadcastHashJoin [codegen id : 27] +Left keys [2]: [ss_item_sk#120, ss_ticket_number#127] +Right keys [2]: [sr_item_sk#134, sr_ticket_number#135] Join condition: None -(121) Project [codegen id : 28] -Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -Input [14]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, sr_item_sk#135, sr_ticket_number#136] +(120) Project [codegen id : 27] +Output [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Input [14]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131, sr_item_sk#134, sr_ticket_number#135] -(122) Exchange -Input [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -Arguments: hashpartitioning(ss_item_sk#121, 5), ENSURE_REQUIREMENTS, [id=#138] +(121) Exchange +Input [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Arguments: hashpartitioning(ss_item_sk#120, 5), ENSURE_REQUIREMENTS, [id=#137] -(123) Sort [codegen id : 29] -Input [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -Arguments: [ss_item_sk#121 ASC NULLS FIRST], false, 0 +(122) Sort [codegen id : 28] +Input [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Arguments: [ss_item_sk#120 ASC NULLS FIRST], false, 0 -(124) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#139, sum#140, sum#141, isEmpty#142] +(123) ReusedExchange [Reuses operator id: 28] +Output [4]: [cs_item_sk#138, sum#139, sum#140, isEmpty#141] -(125) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#139, sum#140, sum#141, isEmpty#142] -Keys [1]: [cs_item_sk#139] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#143)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#143))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#39] -Results [3]: [cs_item_sk#139, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#143))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41] +(124) HashAggregate [codegen id : 34] +Input [4]: [cs_item_sk#138, sum#139, sum#140, isEmpty#141] +Keys [1]: [cs_item_sk#138] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#142)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#143 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#144 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#145 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#142))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#143 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#144 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#145 as decimal(9,2)))), DecimalType(9,2), true))#39] +Results [3]: [cs_item_sk#138, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#142))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#143 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#144 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#145 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41] -(126) Filter [codegen id : 35] -Input [3]: [cs_item_sk#139, sale#40, refund#41] +(125) Filter [codegen id : 34] +Input [3]: [cs_item_sk#138, sale#40, refund#41] Condition : (isnotnull(sale#40) AND (cast(sale#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#41)), DecimalType(21,2), true))) -(127) Project [codegen id : 35] -Output [1]: [cs_item_sk#139] -Input [3]: [cs_item_sk#139, sale#40, refund#41] +(126) Project [codegen id : 34] +Output [1]: [cs_item_sk#138] +Input [3]: [cs_item_sk#138, sale#40, refund#41] -(128) Sort [codegen id : 35] -Input [1]: [cs_item_sk#139] -Arguments: [cs_item_sk#139 ASC NULLS FIRST], false, 0 +(127) Sort [codegen id : 34] +Input [1]: [cs_item_sk#138] +Arguments: [cs_item_sk#138 ASC NULLS FIRST], false, 0 -(129) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#121] -Right keys [1]: [cs_item_sk#139] +(128) SortMergeJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#120] +Right keys [1]: [cs_item_sk#138] Join condition: None -(130) Project [codegen id : 51] -Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, cs_item_sk#139] +(129) Project [codegen id : 50] +Output [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131, cs_item_sk#138] -(131) ReusedExchange [Reuses operator id: 191] -Output [2]: [d_date_sk#147, d_year#148] +(130) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#146, d_year#147] -(132) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#132] -Right keys [1]: [d_date_sk#147] +(131) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#131] +Right keys [1]: [d_date_sk#146] Join condition: None -(133) Project [codegen id : 51] -Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148] -Input [13]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, d_date_sk#147, d_year#148] +(132) Project [codegen id : 50] +Output [11]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147] +Input [13]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131, d_date_sk#146, d_year#147] -(134) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] +(133) ReusedExchange [Reuses operator id: 41] +Output [3]: [s_store_sk#148, s_store_name#149, s_zip#150] -(135) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#126] -Right keys [1]: [s_store_sk#149] +(134) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_store_sk#125] +Right keys [1]: [s_store_sk#148] Join condition: None -(136) Project [codegen id : 51] -Output [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151] -Input [14]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_sk#149, s_store_name#150, s_zip#151] +(135) Project [codegen id : 50] +Output [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150] +Input [14]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_sk#148, s_store_name#149, s_zip#150] -(137) ReusedExchange [Reuses operator id: 47] -Output [6]: [c_customer_sk#152, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157] +(136) ReusedExchange [Reuses operator id: 47] +Output [6]: [c_customer_sk#151, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156] -(138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#122] -Right keys [1]: [c_customer_sk#152] +(137) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_customer_sk#121] +Right keys [1]: [c_customer_sk#151] Join condition: None -(139) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157] -Input [18]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_customer_sk#152, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157] +(138) Project [codegen id : 50] +Output [16]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156] +Input [18]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_customer_sk#151, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156] -(140) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#158, d_year#159] +(139) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#157, d_year#158] -(141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#157] -Right keys [1]: [d_date_sk#158] +(140) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [c_first_sales_date_sk#156] +Right keys [1]: [d_date_sk#157] Join condition: None -(142) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, d_year#159] -Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157, d_date_sk#158, d_year#159] +(141) Project [codegen id : 50] +Output [16]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, d_year#158] +Input [18]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, c_first_sales_date_sk#156, d_date_sk#157, d_year#158] -(143) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#160, d_year#161] +(142) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#159, d_year#160] -(144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#156] -Right keys [1]: [d_date_sk#160] +(143) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [c_first_shipto_date_sk#155] +Right keys [1]: [d_date_sk#159] Join condition: None -(145) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161] -Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, d_year#159, d_date_sk#160, d_year#161] +(144) Project [codegen id : 50] +Output [16]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160] +Input [18]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, c_first_shipto_date_sk#155, d_year#158, d_date_sk#159, d_year#160] -(146) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#162, cd_marital_status#163] +(145) ReusedExchange [Reuses operator id: 62] +Output [2]: [cd_demo_sk#161, cd_marital_status#162] -(147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#123] -Right keys [1]: [cd_demo_sk#162] +(146) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_cdemo_sk#122] +Right keys [1]: [cd_demo_sk#161] Join condition: None -(148) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_marital_status#163] -Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_demo_sk#162, cd_marital_status#163] +(147) Project [codegen id : 50] +Output [16]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, cd_marital_status#162] +Input [18]: [ss_item_sk#120, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, cd_demo_sk#161, cd_marital_status#162] -(149) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#164, cd_marital_status#165] +(148) ReusedExchange [Reuses operator id: 62] +Output [2]: [cd_demo_sk#163, cd_marital_status#164] -(150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#153] -Right keys [1]: [cd_demo_sk#164] -Join condition: NOT (cd_marital_status#163 = cd_marital_status#165) +(149) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [c_current_cdemo_sk#152] +Right keys [1]: [cd_demo_sk#163] +Join condition: NOT (cd_marital_status#162 = cd_marital_status#164) -(151) Project [codegen id : 51] -Output [14]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161] -Input [18]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_marital_status#163, cd_demo_sk#164, cd_marital_status#165] +(150) Project [codegen id : 50] +Output [14]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160] +Input [18]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_cdemo_sk#152, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, cd_marital_status#162, cd_demo_sk#163, cd_marital_status#164] -(152) ReusedExchange [Reuses operator id: 71] -Output [1]: [p_promo_sk#166] +(151) ReusedExchange [Reuses operator id: 71] +Output [1]: [p_promo_sk#165] -(153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#127] -Right keys [1]: [p_promo_sk#166] +(152) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_promo_sk#126] +Right keys [1]: [p_promo_sk#165] Join condition: None -(154) Project [codegen id : 51] -Output [13]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161] -Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, p_promo_sk#166] +(153) Project [codegen id : 50] +Output [13]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160] +Input [15]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_promo_sk#126, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, p_promo_sk#165] -(155) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#167, hd_income_band_sk#168] +(154) ReusedExchange [Reuses operator id: 77] +Output [2]: [hd_demo_sk#166, hd_income_band_sk#167] -(156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#124] -Right keys [1]: [hd_demo_sk#167] +(155) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_hdemo_sk#123] +Right keys [1]: [hd_demo_sk#166] Join condition: None -(157) Project [codegen id : 51] -Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168] -Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_demo_sk#167, hd_income_band_sk#168] +(156) Project [codegen id : 50] +Output [13]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167] +Input [15]: [ss_item_sk#120, ss_hdemo_sk#123, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, hd_demo_sk#166, hd_income_band_sk#167] -(158) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#169, hd_income_band_sk#170] +(157) ReusedExchange [Reuses operator id: 77] +Output [2]: [hd_demo_sk#168, hd_income_band_sk#169] -(159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#154] -Right keys [1]: [hd_demo_sk#169] +(158) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [c_current_hdemo_sk#153] +Right keys [1]: [hd_demo_sk#168] Join condition: None -(160) Project [codegen id : 51] -Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170] -Input [15]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_demo_sk#169, hd_income_band_sk#170] +(159) Project [codegen id : 50] +Output [13]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169] +Input [15]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_hdemo_sk#153, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_demo_sk#168, hd_income_band_sk#169] -(161) ReusedExchange [Reuses operator id: 86] -Output [5]: [ca_address_sk#171, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175] +(160) ReusedExchange [Reuses operator id: 86] +Output [5]: [ca_address_sk#170, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174] -(162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#125] -Right keys [1]: [ca_address_sk#171] +(161) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_addr_sk#124] +Right keys [1]: [ca_address_sk#170] Join condition: None -(163) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175] -Input [18]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_address_sk#171, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175] +(162) Project [codegen id : 50] +Output [16]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174] +Input [18]: [ss_item_sk#120, ss_addr_sk#124, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_address_sk#170, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174] -(164) ReusedExchange [Reuses operator id: 86] -Output [5]: [ca_address_sk#176, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] +(163) ReusedExchange [Reuses operator id: 86] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -(165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#155] -Right keys [1]: [ca_address_sk#176] +(164) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [c_current_addr_sk#154] +Right keys [1]: [ca_address_sk#175] Join condition: None -(166) Project [codegen id : 51] -Output [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] -Input [21]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_address_sk#176, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] +(165) Project [codegen id : 50] +Output [19]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [21]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, c_current_addr_sk#154, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -(167) ReusedExchange [Reuses operator id: 95] -Output [1]: [ib_income_band_sk#181] +(166) ReusedExchange [Reuses operator id: 95] +Output [1]: [ib_income_band_sk#180] -(168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#168] -Right keys [1]: [ib_income_band_sk#181] +(167) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [hd_income_band_sk#167] +Right keys [1]: [ib_income_band_sk#180] Join condition: None -(169) Project [codegen id : 51] -Output [18]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] -Input [20]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, ib_income_band_sk#181] +(168) Project [codegen id : 50] +Output [18]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [20]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#167, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ib_income_band_sk#180] -(170) ReusedExchange [Reuses operator id: 95] -Output [1]: [ib_income_band_sk#182] +(169) ReusedExchange [Reuses operator id: 95] +Output [1]: [ib_income_band_sk#181] -(171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#170] -Right keys [1]: [ib_income_band_sk#182] +(170) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [hd_income_band_sk#169] +Right keys [1]: [ib_income_band_sk#181] Join condition: None -(172) Project [codegen id : 51] -Output [17]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] -Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, ib_income_band_sk#182] +(171) Project [codegen id : 50] +Output [17]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [19]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, hd_income_band_sk#169, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ib_income_band_sk#181] -(173) ReusedExchange [Reuses operator id: 105] -Output [2]: [i_item_sk#183, i_product_name#184] +(172) ReusedExchange [Reuses operator id: 105] +Output [2]: [i_item_sk#182, i_product_name#183] -(174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#121] -Right keys [1]: [i_item_sk#183] +(173) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#120] +Right keys [1]: [i_item_sk#182] Join condition: None -(175) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, d_year#159, d_year#161, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184] -Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184] - -(176) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, d_year#159, d_year#161, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184] -Keys [15]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#129)), partial_sum(UnscaledValue(ss_list_price#130)), partial_sum(UnscaledValue(ss_coupon_amt#131))] -Aggregate Attributes [4]: [count#91, sum#185, sum#186, sum#187] -Results [19]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161, count#95, sum#188, sum#189, sum#190] - -(177) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161, count#95, sum#188, sum#189, sum#190] -Keys [15]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#129)), sum(UnscaledValue(ss_list_price#130)), sum(UnscaledValue(ss_coupon_amt#131))] -Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#129))#100, sum(UnscaledValue(ss_list_price#130))#101, sum(UnscaledValue(ss_coupon_amt#131))#102] -Results [8]: [i_item_sk#183 AS item_sk#191, s_store_name#150 AS store_name#192, s_zip#151 AS store_zip#193, d_year#148 AS syear#194, count(1)#99 AS cnt#195, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#129))#100,17,2) AS s1#196, MakeDecimal(sum(UnscaledValue(ss_list_price#130))#101,17,2) AS s2#197, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#131))#102,17,2) AS s3#198] - -(178) Exchange -Input [8]: [item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198] -Arguments: hashpartitioning(item_sk#191, store_name#192, store_zip#193, 5), ENSURE_REQUIREMENTS, [id=#199] - -(179) Sort [codegen id : 52] -Input [8]: [item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198] -Arguments: [item_sk#191 ASC NULLS FIRST, store_name#192 ASC NULLS FIRST, store_zip#193 ASC NULLS FIRST], false, 0 - -(180) SortMergeJoin [codegen id : 53] +(174) Project [codegen id : 50] +Output [18]: [ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, d_year#158, d_year#160, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, i_item_sk#182, i_product_name#183] +Input [19]: [ss_item_sk#120, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, s_store_name#149, s_zip#150, d_year#158, d_year#160, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, i_item_sk#182, i_product_name#183] + +(175) HashAggregate [codegen id : 50] +Input [18]: [ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, d_year#147, d_year#158, d_year#160, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, i_item_sk#182, i_product_name#183] +Keys [15]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#128)), partial_sum(UnscaledValue(ss_list_price#129)), partial_sum(UnscaledValue(ss_coupon_amt#130))] +Aggregate Attributes [4]: [count#91, sum#184, sum#185, sum#186] +Results [19]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160, count#95, sum#187, sum#188, sum#189] + +(176) HashAggregate [codegen id : 50] +Input [19]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160, count#95, sum#187, sum#188, sum#189] +Keys [15]: [i_product_name#183, i_item_sk#182, s_store_name#149, s_zip#150, ca_street_number#171, ca_street_name#172, ca_city#173, ca_zip#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, d_year#147, d_year#158, d_year#160] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#128)), sum(UnscaledValue(ss_list_price#129)), sum(UnscaledValue(ss_coupon_amt#130))] +Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#128))#100, sum(UnscaledValue(ss_list_price#129))#101, sum(UnscaledValue(ss_coupon_amt#130))#102] +Results [8]: [i_item_sk#182 AS item_sk#190, s_store_name#149 AS store_name#191, s_zip#150 AS store_zip#192, d_year#147 AS syear#193, count(1)#99 AS cnt#194, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#128))#100,17,2) AS s1#195, MakeDecimal(sum(UnscaledValue(ss_list_price#129))#101,17,2) AS s2#196, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#130))#102,17,2) AS s3#197] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#190, store_name#191, store_zip#192, syear#193, cnt#194, s1#195, s2#196, s3#197] +Arguments: [item_sk#190 ASC NULLS FIRST, store_name#191 ASC NULLS FIRST, store_zip#192 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] Left keys [3]: [item_sk#104, store_name#105, store_zip#106] -Right keys [3]: [item_sk#191, store_name#192, store_zip#193] -Join condition: (cnt#195 <= cnt#116) +Right keys [3]: [item_sk#190, store_name#191, store_zip#192] +Join condition: (cnt#194 <= cnt#116) -(181) Project [codegen id : 53] -Output [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195] -Input [25]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198] +(179) Project [codegen id : 51] +Output [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#195, s2#196, s3#197, syear#193, cnt#194] +Input [25]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, item_sk#190, store_name#191, store_zip#192, syear#193, cnt#194, s1#195, s2#196, s3#197] -(182) Exchange -Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195] -Arguments: rangepartitioning(product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#195 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#196 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#200] +(180) Exchange +Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#195, s2#196, s3#197, syear#193, cnt#194] +Arguments: rangepartitioning(product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#194 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#195 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#198] -(183) Sort [codegen id : 54] -Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195] -Arguments: [product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#195 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#196 ASC NULLS FIRST], true, 0 +(181) Sort [codegen id : 52] +Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#195, s2#196, s3#197, syear#193, cnt#194] +Arguments: [product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#194 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#195 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (187) -+- * Filter (186) - +- * ColumnarToRow (185) - +- Scan parquet default.date_dim (184) +BroadcastExchange (185) ++- * Filter (184) + +- * ColumnarToRow (183) + +- Scan parquet default.date_dim (182) -(184) Scan parquet default.date_dim +(182) Scan parquet default.date_dim Output [2]: [d_date_sk#42, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(185) ColumnarToRow [codegen id : 1] +(183) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#42, d_year#43] -(186) Filter [codegen id : 1] +(184) Filter [codegen id : 1] Input [2]: [d_date_sk#42, d_year#43] Condition : ((isnotnull(d_year#43) AND (d_year#43 = 1999)) AND isnotnull(d_date_sk#42)) -(187) BroadcastExchange +(185) BroadcastExchange Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] -Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#132 IN dynamicpruning#133 -BroadcastExchange (191) -+- * Filter (190) - +- * ColumnarToRow (189) - +- Scan parquet default.date_dim (188) +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 +BroadcastExchange (189) ++- * Filter (188) + +- * ColumnarToRow (187) + +- Scan parquet default.date_dim (186) -(188) Scan parquet default.date_dim -Output [2]: [d_date_sk#147, d_year#148] +(186) Scan parquet default.date_dim +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(189) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#147, d_year#148] +(187) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#146, d_year#147] -(190) Filter [codegen id : 1] -Input [2]: [d_date_sk#147, d_year#148] -Condition : ((isnotnull(d_year#148) AND (d_year#148 = 2000)) AND isnotnull(d_date_sk#147)) +(188) Filter [codegen id : 1] +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) -(191) BroadcastExchange -Input [2]: [d_date_sk#147, d_year#148] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202] +(189) BroadcastExchange +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 6917f8f6c6e2d..716aaa2663630 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,289 +1,283 @@ -WholeStageCodegen (54) +WholeStageCodegen (52) Sort [product_name,store_name,cnt,s1,s1] InputAdapter Exchange [product_name,store_name,cnt,s1,s1] #1 - WholeStageCodegen (53) + WholeStageCodegen (51) Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter - WholeStageCodegen (26) + WholeStageCodegen (25) Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (25) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Exchange [ss_item_sk] #2 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + Project [sr_item_sk,sr_ticket_number] + Filter [sr_item_sk,sr_ticket_number] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - Project [sr_item_sk,sr_ticket_number] - Filter [sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] - InputAdapter - Exchange [cs_item_sk] #6 - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] - Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (9) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #5 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk,cs_order_number] InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #7 - WholeStageCodegen (4) - Project [cs_item_sk,cs_order_number,cs_ext_list_price] - Filter [cs_item_sk,cs_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + Exchange [cs_item_sk,cs_order_number] #6 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_ext_list_price] + Filter [cs_item_sk,cs_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (7) - Sort [cr_item_sk,cr_order_number] - InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 - WholeStageCodegen (6) - Project [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - Filter [cr_item_sk,cr_order_number] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + Exchange [cr_item_sk,cr_order_number] #7 + WholeStageCodegen (6) + Project [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + Filter [cr_item_sk,cr_order_number] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [s_store_sk,s_store_name,s_zip] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + Scan parquet default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (12) + Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (11) - Filter [s_store_sk,s_store_name,s_zip] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_name,s_zip] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (13) + Filter [d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (12) - Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (13) - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #10 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + Filter [cd_demo_sk,cd_marital_status] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (15) - Filter [cd_demo_sk,cd_marital_status] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #11 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (17) + Filter [p_promo_sk] + ColumnarToRow InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + Scan parquet default.promotion [p_promo_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (18) + Filter [hd_demo_sk,hd_income_band_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (17) - Filter [p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (18) - Filter [hd_demo_sk,hd_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_income_band_sk] + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (20) + Filter [ca_address_sk] + ColumnarToRow InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (20) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (22) - Filter [ib_income_band_sk] - ColumnarToRow - InputAdapter - Scan parquet default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (24) - Project [i_item_sk,i_product_name] - Filter [i_current_price,i_color,i_item_sk] + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (22) + Filter [ib_income_band_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] + Scan parquet default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #15 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (24) + Project [i_item_sk,i_product_name] + Filter [i_current_price,i_color,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter - WholeStageCodegen (52) + WholeStageCodegen (50) Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (51) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SortMergeJoin [ss_item_sk,cs_item_sk] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (28) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (29) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #19 - WholeStageCodegen (28) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Exchange [ss_item_sk] #17 + WholeStageCodegen (27) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #18 + WholeStageCodegen (26) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #19 + WholeStageCodegen (1) + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + Project [sr_item_sk,sr_ticket_number] + Filter [sr_item_sk,sr_ticket_number] + ColumnarToRow InputAdapter - BroadcastExchange #20 - WholeStageCodegen (27) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - Project [sr_item_sk,sr_ticket_number] - Filter [sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (35) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (34) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #5 + InputAdapter + ReusedExchange [d_date_sk,d_year] #19 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #8 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #9 + InputAdapter + ReusedExchange [d_date_sk,d_year] #10 + InputAdapter + ReusedExchange [d_date_sk,d_year] #10 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #11 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #11 + InputAdapter + ReusedExchange [p_promo_sk] #12 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 + InputAdapter + ReusedExchange [ib_income_band_sk] #15 + InputAdapter + ReusedExchange [ib_income_band_sk] #15 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #16 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index 0425be6f9a79e..10dc5ca4dc297 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -135,4 +135,33 @@ class EnsureRequirementsSuite extends SharedSparkSession { }.size == 2) } } + + test("should have no shuffle when clustering keys are subset of join keys") { + val plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) + val plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprB :: Nil, 5)) + + val smjExec1 = SortMergeJoinExec( + exprA :: exprB :: Nil, exprB :: exprC :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec1) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprB, exprC)) + case other => fail(other.toString) + } + + val smjExec2 = SortMergeJoinExec( + exprB :: exprA :: Nil, exprC :: exprB :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec2) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprB, exprA)) + assert(rightKeys === Seq(exprC, exprB)) + case other => fail(other.toString) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index d90c8732ea287..be9d1b0e179fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -773,7 +773,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti // join predicates is a super set of child's partitioning columns val bucketedTableTestSpec1 = - BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), numPartitions = 1) + BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), + numPartitions = 1, expectedShuffle = false) testBucketing( bucketedTableTestSpecLeft = bucketedTableTestSpec1, bucketedTableTestSpecRight = bucketedTableTestSpec1, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 8c830d375d9f4..a24e76f81b4aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -585,7 +585,7 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { // Verify the query plan assert(query.lastExecution.executedPlan.collect { case j @ StreamingSymmetricHashJoinExec(_, _, _, _, _, _, _, _, - _: ShuffleExchangeExec, ShuffleExchangeExec(_, _: ShuffleExchangeExec, _)) => j + _: ShuffleExchangeExec, _: ShuffleExchangeExec) => j }.size == 1) }) } From 5b6e371418272a3147de7d8900ddc61252271d59 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 17 Jun 2021 16:38:08 -0700 Subject: [PATCH 02/29] implement requirement (no tests yet) --- .../plans/physical/partitioning.scala | 231 ++++++++++++------ .../sql/catalyst/DistributionSuite.scala | 199 +-------------- .../exchange/EnsureRequirements.scala | 48 ++-- 3 files changed, 197 insertions(+), 281 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index b79288438753b..c33105a34ed8f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -149,22 +149,17 @@ trait Partitioning { } /** - * Returns true iff this partitioning is compatible with `other`. If two [[Partitioning]]s can - * satisfy their respective required distribution (via [[satisfies]]), and are compatible with - * each other, then their partitions are considered to be co-partitioned, which will allow Spark - * to eliminate data shuffle whenever necessary. - * - * Note: implementor should make sure the method satisfies the equivalence relation, that is, - * the implementation should be reflexive, symmetric and transitive. + * Only return non-empty if the requirement can be used to repartition another side to match + * the distribution of this side. */ - final def isCompatibleWith( - distribution: Distribution, - other: Partitioning, - otherDistribution: Distribution): Boolean = other match { - case PartitioningCollection(others) => - others.exists(_.isCompatibleWith(otherDistribution, this, distribution)) - case _ => isCompatibleWith0(distribution, other, otherDistribution) - } + final def createRequirement(distribution: Distribution): Option[Requirement] = + distribution match { + case clustered: ClusteredDistribution => + createRequirement0(clustered) + case _ => + throw new IllegalStateException(s"Unexpected distribution: " + + s"${distribution.getClass.getSimpleName}") + } /** * The actual method that defines whether this [[Partitioning]] can satisfy the given @@ -180,14 +175,8 @@ trait Partitioning { case _ => false } - /** - * The actual method that defines whether this [[Partitioning]] is compatible with `other`. In - * default this always return false. - */ - protected def isCompatibleWith0( - distribution: Distribution, - other: Partitioning, - otherDistribution: Distribution): Boolean = false + protected def createRequirement0(distribution: ClusteredDistribution): Option[Requirement] = + None } case class UnknownPartitioning(numPartitions: Int) extends Partitioning @@ -207,13 +196,8 @@ case object SinglePartition extends Partitioning { case _ => true } - override def isCompatibleWith0( - distribution: Distribution, - other: Partitioning, - otherDistribution: Distribution): Boolean = other match { - case SinglePartition => true - case _ => false - } + override protected def createRequirement0( + distribution: ClusteredDistribution): Option[Requirement] = Some(SinglePartitionRequirement) } /** @@ -238,43 +222,8 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } - override def isCompatibleWith0( - distribution: Distribution, - other: Partitioning, - otherDistribution: Distribution): Boolean = (distribution, otherDistribution) match { - case (thisDist: ClusteredDistribution, thatDist: ClusteredDistribution) => - // For each expression in the `HashPartitioning` that has occurrences in - // `ClusteredDistribution`, returns a mapping from its index in the partitioning to the - // indexes where it appears in the distribution. - // For instance, if `partitioning` is `[a, b]` and `distribution is `[a, a, b]`, then the - // result mapping could be `{ 0 -> (0, 1), 1 -> (2) }`. - def indexMap( - distribution: ClusteredDistribution, - partitioning: HashPartitioning): mutable.Map[Int, mutable.BitSet] = { - val result = mutable.Map.empty[Int, mutable.BitSet] - val expressionToIndex = partitioning.expressions.zipWithIndex.toMap - distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyIdx) => - expressionToIndex.find { case (partKey, _) => partKey.semanticEquals(distKey) }.forall { - case (_, partIdx) => - result.getOrElseUpdate(partIdx, mutable.BitSet.empty).add(distKeyIdx) - } - } - result - } - - other match { - case that @ HashPartitioning(_, _) => - // we need to check: - // 1. both partitioning have the same number of expressions - // 2. each corresponding expression in both partitioning is used in the same positions - // of the corresponding distribution. - this.expressions.length == that.expressions.length && - indexMap(thisDist, this) == indexMap(thatDist, that) - case _ => - false - } - case _ => - false + override def createRequirement0(distribution: ClusteredDistribution): Option[Requirement] = { + Some(HashRequirement(this, distribution)) } /** @@ -377,11 +326,16 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def satisfies0(required: Distribution): Boolean = partitionings.exists(_.satisfies(required)) - override def isCompatibleWith0( - distribution: Distribution, - other: Partitioning, - otherDistribution: Distribution): Boolean = - partitionings.exists(_.isCompatibleWith(distribution, other, otherDistribution)) + override def createRequirement0(distribution: ClusteredDistribution): Option[Requirement] = { + val eligible = partitionings + .filter(_.satisfies(distribution)) + .flatMap(_.createRequirement(distribution)) + if (eligible.nonEmpty) { + Some(RequirementCollection(eligible)) + } else { + None + } + } override def toString: String = { partitionings.map(_.toString).mkString("(", " or ", ")") @@ -405,3 +359,136 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { case _ => false } } + +// --------------------------------------------------------- +// ------------------ Requirements ------------------------- +// --------------------------------------------------------- + +/** + * This specifies that, when a operator has more than one children where each of which has + * its own partitioning and required distribution, the requirement for the other children to be + * co-partitioned with the current child. + */ +trait Requirement extends Ordered[Requirement] { + /** + * Returns true iff this requirement is compatible with the other [[Partitioning]] and + * clustering expressions (e.g., from [[ClusteredDistribution]]). + * + * If compatible, then it means that the data partitioning from the requirement can be seen as + * co-partitioned with the `otherPartitioning`, and therefore no shuffle is required when + * joining the two sides. + */ + def isCompatibleWith(otherPartitioning: Partitioning, otherClustering: Seq[Expression]): Boolean + + /** + * Create a partitioning that can be used to re-partitioned the other side whose required + * distribution is specified via `clustering`. + * + * Note: this will only be called after `isCompatibleWith` returns true on the side where the + * `clustering` is returned from. + */ + def createPartitioning(clustering: Seq[Expression]): Partitioning +} + +case object SinglePartitionRequirement extends Requirement { + override def isCompatibleWith( + otherPartitioning: Partitioning, + otherClustering: Seq[Expression]): Boolean = { + otherPartitioning.numPartitions == 1 + } + + override def createPartitioning(clustering: Seq[Expression]): Partitioning = + SinglePartition + + override def compare(that: Requirement): Int = that match { + case SinglePartitionRequirement => + 0 + case HashRequirement(partitioning, _) => + 1.compare(partitioning.numPartitions) + case RequirementCollection(requirements) => + requirements.map(compare).min + } +} + +case class HashRequirement( + partitioning: HashPartitioning, + distribution: ClusteredDistribution) extends Requirement { + private lazy val matchingIndexes = indexMap(distribution.clustering, partitioning.expressions) + + override def isCompatibleWith( + otherPartitioning: Partitioning, + otherClustering: Seq[Expression]): Boolean = otherPartitioning match { + case SinglePartition => + partitioning.numPartitions == 1 + case HashPartitioning(expressions, _) => + // we need to check: + // 1. both partitioning have the same number of expressions + // 2. each corresponding expression in both partitioning is used in the same positions + // of the corresponding distribution. + partitioning.expressions.length == expressions.length && + matchingIndexes == indexMap(otherClustering, expressions) + case PartitioningCollection(partitionings) => + partitionings.exists(isCompatibleWith(_, otherClustering)) + case _ => + false + } + + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { + val exprs = clustering + .zipWithIndex + .filter(x => matchingIndexes.keySet.contains(x._2)) + .map(_._1) + HashPartitioning(exprs, partitioning.numPartitions) + } + + override def compare(that: Requirement): Int = that match { + case SinglePartitionRequirement => + partitioning.numPartitions.compare(1) + case HashRequirement(otherPartitioning, _) => + if (partitioning.numPartitions != otherPartitioning.numPartitions) { + partitioning.numPartitions.compare(otherPartitioning.numPartitions) + } else { + partitioning.expressions.length.compare(otherPartitioning.expressions.length) + } + case RequirementCollection(requirements) => + // pick the best requirement in the other collection + requirements.map(compare).min + } + + // For each expression in the `HashPartitioning` that has occurrences in + // `ClusteredDistribution`, returns a mapping from its index in the partitioning to the + // indexes where it appears in the distribution. + // For instance, if `partitioning` is `[a, b]` and `distribution is `[a, a, b]`, then the + // result mapping could be `{ 0 -> (0, 1), 1 -> (2) }`. + private def indexMap( + clustering: Seq[Expression], + expressions: Seq[Expression]): mutable.Map[Int, mutable.BitSet] = { + val result = mutable.Map.empty[Int, mutable.BitSet] + val expressionToIndex = expressions.zipWithIndex.toMap + clustering.zipWithIndex.foreach { case (distKey, distKeyIdx) => + expressionToIndex.find { case (partKey, _) => partKey.semanticEquals(distKey) }.forall { + case (_, partIdx) => + result.getOrElseUpdate(partIdx, mutable.BitSet.empty).add(distKeyIdx) + } + } + result + } +} + +case class RequirementCollection(requirements: Seq[Requirement]) extends Requirement { + override def isCompatibleWith( + otherPartitioning: Partitioning, + otherClustering: Seq[Expression]): Boolean = { + requirements.exists(_.isCompatibleWith(otherPartitioning, otherClustering)) + } + + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { + // choose the best requirement (e.g., maximum shuffle parallelism, min shuffle data size, etc) + // from the collection and use that to repartition the other sides + requirements.max.createPartitioning(clustering) + } + + override def compare(that: Requirement): Int = { + requirements.map(_.compare(that)).max + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 8b7f9ac41c79d..a77560cf3faac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -31,44 +31,18 @@ class DistributionSuite extends SparkFunSuite { if (inputPartitioning.satisfies(requiredDistribution) != satisfied) { fail( s""" - |== Input Partitioning == - |$inputPartitioning - |== Required Distribution == - |$requiredDistribution - |== Does input partitioning satisfy required distribution? == - |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} - """.stripMargin) - } - } + |== Input Partitioning == + |$inputPartitioning - protected def checkCompatible( - left: Partitioning, - right: Partitioning, - compatible: Boolean, - leftDistribution: Distribution = UnspecifiedDistribution, - rightDistribution: Distribution = UnspecifiedDistribution): Unit = { - val actual = left.isCompatibleWith(leftDistribution, right, rightDistribution) - if (actual != compatible) { - fail( - s""" - |== Left Partitioning == - |$left - |== Right Partitioning == - |$right - |== Is left partitioning compatible with right partitioning? == - |Expected $compatible but got $actual - |""".stripMargin) - } - } + |== Required Distri + + |$requiredDistribution - protected def checkPartitionCollectionCompatible( - left: Partitioning, - right: Partitioning, - compatible: Boolean, - leftDistribution: Distribution = UnspecifiedDistribution, - rightDistribution: Distribution = UnspecifiedDistribution): Unit = { - checkCompatible(left, right, compatible, leftDistribution, rightDistribution) - checkCompatible(right, left, compatible, rightDistribution, leftDistribution) + |== Does input partitioning satisfy require + ution? == + |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} + """.stripMargin) + } } test("UnspecifiedDistribution and AllTuples") { @@ -295,157 +269,4 @@ class DistributionSuite extends SparkFunSuite { ClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), false) } - - test("Compatibility: SinglePartition and HashPartitioning") { - checkCompatible( - SinglePartition, - SinglePartition, - compatible = true) - - checkCompatible( - HashPartitioning(Seq($"a", $"b", $"c"), 4), - HashPartitioning(Seq($"a", $"b", $"c"), 4), - compatible = true, - ClusteredDistribution(Seq($"a", $"b", $"c")), - ClusteredDistribution(Seq($"a", $"b", $"c"))) - - checkCompatible( - HashPartitioning(Seq($"a", $"c"), 4), - HashPartitioning(Seq($"a", $"c"), 8), - compatible = true, - ClusteredDistribution(Seq($"a", $"b", $"c")), - ClusteredDistribution(Seq($"a", $"b", $"c"))) - - checkCompatible( - HashPartitioning(Seq($"a", $"b"), 4), - HashPartitioning(Seq($"b", $"a"), 8), - compatible = true, - ClusteredDistribution(Seq($"a", $"b")), - ClusteredDistribution(Seq($"b", $"a"))) - - checkCompatible( - HashPartitioning(Seq($"a", $"b"), 4), - HashPartitioning(Seq($"b", $"a"), 8), - compatible = true, - ClusteredDistribution(Seq($"c", $"a", $"b")), - ClusteredDistribution(Seq($"d", $"b", $"a"))) - - checkCompatible( - HashPartitioning(Seq($"a", $"b"), 4), - HashPartitioning(Seq($"a", $"b"), 8), - compatible = true, - ClusteredDistribution(Seq($"a", $"a", $"b")), - ClusteredDistribution(Seq($"a", $"a", $"b"))) - - // negative cases - - checkCompatible( - HashPartitioning(Seq($"a"), 4), - HashPartitioning(Seq($"b"), 4), - compatible = false, - ClusteredDistribution(Seq($"a", $"b")), - ClusteredDistribution(Seq($"a", $"b"))) - - checkCompatible( - HashPartitioning(Seq($"a", $"b"), 4), - HashPartitioning(Seq($"b", $"b"), 4), - compatible = false, - ClusteredDistribution(Seq($"a", $"b")), - ClusteredDistribution(Seq($"b", $"a"))) - - checkCompatible( - HashPartitioning(Seq($"a", $"b"), 4), - HashPartitioning(Seq($"a", $"b"), 4), - compatible = false, - ClusteredDistribution(Seq($"a", $"a", $"b")), - ClusteredDistribution(Seq($"a", $"b", $"b"))) - } - - test("Compatibility: PartitionCollection") { - checkPartitionCollectionCompatible( - HashPartitioning(Seq($"a"), 4), - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), - compatible = true, - ClusteredDistribution(Seq($"a")), - ClusteredDistribution(Seq($"a"))) - - checkPartitionCollectionCompatible( - HashPartitioning(Seq($"a"), 4), - PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4), HashPartitioning(Seq($"a"), 4))), - compatible = true, - ClusteredDistribution(Seq($"a")), - ClusteredDistribution(Seq($"a", $"b"))) - - checkPartitionCollectionCompatible( - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), - compatible = true, - ClusteredDistribution(Seq($"a")), - ClusteredDistribution(Seq($"a"))) - - checkPartitionCollectionCompatible( - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), - compatible = true, - ClusteredDistribution(Seq($"a", $"b")), - ClusteredDistribution(Seq($"a", $"b"))) - - checkPartitionCollectionCompatible( - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), - PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4), HashPartitioning(Seq($"a"), 4))), - compatible = true, - ClusteredDistribution(Seq($"a", $"b")), - ClusteredDistribution(Seq($"a", $"b"))) - - checkPartitionCollectionCompatible( - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4))), - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), - compatible = true, - ClusteredDistribution(Seq($"a")), - ClusteredDistribution(Seq($"a", $"b"))) - - checkPartitionCollectionCompatible( - PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4), HashPartitioning(Seq($"a"), 4))), - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"c"), 4))), - compatible = true, - ClusteredDistribution(Seq($"b", $"a")), - ClusteredDistribution(Seq($"a", $"c"))) - - // negative cases - - checkPartitionCollectionCompatible( - HashPartitioning(Seq($"a"), 4), - PartitioningCollection(Seq(HashPartitioning(Seq($"b"), 4))), - compatible = false, - ClusteredDistribution(Seq($"a", $"b")), - ClusteredDistribution(Seq($"a", $"b"))) - - checkCompatible( - PartitioningCollection(Seq(HashPartitioning(Seq($"a"), 4), HashPartitioning(Seq($"b"), 4))), - PartitioningCollection(Seq(HashPartitioning(Seq($"c"), 4))), - compatible = false, - ClusteredDistribution(Seq($"a", $"b", $"c")), - ClusteredDistribution(Seq($"a", $"b", $"c"))) - } - - test("Compatibility: Others") { - val partitionings: Seq[Partitioning] = Seq(UnknownPartitioning(1), - BroadcastPartitioning(IdentityBroadcastMode), - RoundRobinPartitioning(10), - RangePartitioning(Seq($"a".asc), 10), - PartitioningCollection(Seq(UnknownPartitioning(1))) - ) - - for (i <- partitionings.indices) { - for (j <- partitionings.indices) { - checkCompatible(partitionings(i), partitionings(j), compatible = false) - } - } - - // should always return false when comparing with `HashPartitioning` or `SinglePartition` - partitionings.foreach { p => - checkCompatible(p, HashPartitioning(Seq($"a"), 10), compatible = false) - checkCompatible(p, SinglePartition, compatible = false) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index d8d76910d2c6a..c47d856608676 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -73,33 +73,41 @@ case class EnsureRequirements( case _ => true }.map(_._2) - // Assuming equivalence relation in partitioning compatibility check - val allCompatible = childrenIndexes - .map(i => (children(i).outputPartitioning, requiredChildDistributions(i))) - .sliding(2).map { - case Seq(_) => true - case Seq((ap, ad), (bp, bd)) => ap.isCompatibleWith(ad, bp, bd) - }.forall(_ == true) + // If there are more than one children, we'll need to check partitioning & distribution of them + // and see if we need to insert extra shuffle. + if (childrenIndexes.length > 1 && childrenIndexes.map(requiredChildDistributions(_)) + .forall(_.isInstanceOf[ClusteredDistribution])) { + val childrenWithDistribution = childrenIndexes + .map(i => (children(i).outputPartitioning, + requiredChildDistributions(i).asInstanceOf[ClusteredDistribution])) + val requirements = childrenWithDistribution.flatMap(pd => pd._1.createRequirement(pd._2)) - if (!allCompatible) { - // insert shuffle for all children that are not compatible children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, _), idx) if !childrenIndexes.contains(idx) => child case ((child, dist), _) => - val numPartitions = dist.requiredNumPartitions.getOrElse(conf.numShufflePartitions) - val defaultPartitioning = dist.createPartitioning(numPartitions) - // check if the child's partitioning is already the same as default partitioning, and - // skip the shuffle if so. - // TODO: we should find the "least common" partitioning for all children and use that - if (!child.outputPartitioning.isCompatibleWith(dist, defaultPartitioning, dist)) { - child match { - case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(defaultPartitioning, c, so) - case _ => ShuffleExchangeExec(defaultPartitioning, child) - } + val partitioningOpt = if (requirements.isEmpty) { + // all the children need to be re-shuffled + val numPartitions = dist.requiredNumPartitions.getOrElse(conf.numShufflePartitions) + Some(dist.createPartitioning(numPartitions)) } else { - child + // pick the best candidate from the requirements and use that to re-shuffle other + // children if necessary + val best = requirements.max + val clustering = dist.asInstanceOf[ClusteredDistribution].clustering + if (best.isCompatibleWith(child.outputPartitioning, clustering)) { + None + } else { + Some(requirements.max.createPartitioning(clustering)) + } } + + partitioningOpt.map { p => + child match { + case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(p, c, so) + case _ => ShuffleExchangeExec(p, child) + } + }.getOrElse(child) } } From a6fba50c4793541bc3ef5bd9f49db3640f16695d Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 1 Sep 2021 09:20:33 -0700 Subject: [PATCH 03/29] wip --- .../spark/sql/execution/exchange/EnsureRequirements.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index c47d856608676..04d171f484235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -74,9 +74,10 @@ case class EnsureRequirements( }.map(_._2) // If there are more than one children, we'll need to check partitioning & distribution of them - // and see if we need to insert extra shuffle. + // and see if extra shuffles are necessary. if (childrenIndexes.length > 1 && childrenIndexes.map(requiredChildDistributions(_)) .forall(_.isInstanceOf[ClusteredDistribution])) { + // TODO: can we handle AllTuples together? val childrenWithDistribution = childrenIndexes .map(i => (children(i).outputPartitioning, requiredChildDistributions(i).asInstanceOf[ClusteredDistribution])) @@ -98,7 +99,7 @@ case class EnsureRequirements( if (best.isCompatibleWith(child.outputPartitioning, clustering)) { None } else { - Some(requirements.max.createPartitioning(clustering)) + Some(best.createPartitioning(clustering)) } } From 45c9a051b2ca642fe0c015b784c9d03b3c374256 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 2 Sep 2021 18:54:48 -0700 Subject: [PATCH 04/29] wip --- .../plans/physical/partitioning.scala | 63 ++++++++++--------- 1 file changed, 35 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index c33105a34ed8f..efcbfb23228b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -152,7 +152,7 @@ trait Partitioning { * Only return non-empty if the requirement can be used to repartition another side to match * the distribution of this side. */ - final def createRequirement(distribution: Distribution): Option[Requirement] = + final def createRequirement(distribution: Distribution): Option[ShuffleSpec] = distribution match { case clustered: ClusteredDistribution => createRequirement0(clustered) @@ -175,7 +175,7 @@ trait Partitioning { case _ => false } - protected def createRequirement0(distribution: ClusteredDistribution): Option[Requirement] = + protected def createRequirement0(distribution: ClusteredDistribution): Option[ShuffleSpec] = None } @@ -197,7 +197,7 @@ case object SinglePartition extends Partitioning { } override protected def createRequirement0( - distribution: ClusteredDistribution): Option[Requirement] = Some(SinglePartitionRequirement) + distribution: ClusteredDistribution): Option[ShuffleSpec] = Some(SinglePartitionShuffleSpec$) } /** @@ -222,8 +222,8 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } - override def createRequirement0(distribution: ClusteredDistribution): Option[Requirement] = { - Some(HashRequirement(this, distribution)) + override def createRequirement0(distribution: ClusteredDistribution): Option[ShuffleSpec] = { + Some(HashShuffleSpec(this, distribution)) } /** @@ -326,12 +326,12 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def satisfies0(required: Distribution): Boolean = partitionings.exists(_.satisfies(required)) - override def createRequirement0(distribution: ClusteredDistribution): Option[Requirement] = { + override def createRequirement0(distribution: ClusteredDistribution): Option[ShuffleSpec] = { val eligible = partitionings .filter(_.satisfies(distribution)) .flatMap(_.createRequirement(distribution)) if (eligible.nonEmpty) { - Some(RequirementCollection(eligible)) + Some(ShuffleSpecCollection(eligible)) } else { None } @@ -360,25 +360,32 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { } } -// --------------------------------------------------------- -// ------------------ Requirements ------------------------- -// --------------------------------------------------------- - /** * This specifies that, when a operator has more than one children where each of which has * its own partitioning and required distribution, the requirement for the other children to be * co-partitioned with the current child. */ -trait Requirement extends Ordered[Requirement] { + +/** + * This is used in the scenario where an operator has multiple children (e.g., join), and each of + * which has its own partitioning and required distribution. The spec is mainly used for two things: + * + * 1. Compare with specs from other children and check if they are compatible. When two specs + * are compatible, we can say their data are co-partitioned, and thus will allow Spark to + * eliminate shuffle in operators such as join. + * 2. In case this spec is not compatible with another, create a partitioning that can be used to + * re-partition the other side. + */ +trait ShuffleSpec extends Ordered[ShuffleSpec] { /** - * Returns true iff this requirement is compatible with the other [[Partitioning]] and + * Returns true iff this spec is compatible with the other [[Partitioning]] and * clustering expressions (e.g., from [[ClusteredDistribution]]). * - * If compatible, then it means that the data partitioning from the requirement can be seen as + * A true return value means that the data partitioning from this spec can be seen as * co-partitioned with the `otherPartitioning`, and therefore no shuffle is required when * joining the two sides. */ - def isCompatibleWith(otherPartitioning: Partitioning, otherClustering: Seq[Expression]): Boolean + def isCompatibleWith(other: ShuffleSpec): Boolean /** * Create a partitioning that can be used to re-partitioned the other side whose required @@ -390,7 +397,7 @@ trait Requirement extends Ordered[Requirement] { def createPartitioning(clustering: Seq[Expression]): Partitioning } -case object SinglePartitionRequirement extends Requirement { +case object SinglePartitionShuffleSpec$ extends ShuffleSpec { override def isCompatibleWith( otherPartitioning: Partitioning, otherClustering: Seq[Expression]): Boolean = { @@ -400,19 +407,19 @@ case object SinglePartitionRequirement extends Requirement { override def createPartitioning(clustering: Seq[Expression]): Partitioning = SinglePartition - override def compare(that: Requirement): Int = that match { - case SinglePartitionRequirement => + override def compare(that: ShuffleSpec): Int = that match { + case SinglePartitionShuffleSpec$ => 0 - case HashRequirement(partitioning, _) => + case HashShuffleSpec(partitioning, _) => 1.compare(partitioning.numPartitions) - case RequirementCollection(requirements) => + case ShuffleSpecCollection(requirements) => requirements.map(compare).min } } -case class HashRequirement( +case class HashShuffleSpec( partitioning: HashPartitioning, - distribution: ClusteredDistribution) extends Requirement { + distribution: ClusteredDistribution) extends ShuffleSpec { private lazy val matchingIndexes = indexMap(distribution.clustering, partitioning.expressions) override def isCompatibleWith( @@ -441,16 +448,16 @@ case class HashRequirement( HashPartitioning(exprs, partitioning.numPartitions) } - override def compare(that: Requirement): Int = that match { - case SinglePartitionRequirement => + override def compare(that: ShuffleSpec): Int = that match { + case SinglePartitionShuffleSpec$ => partitioning.numPartitions.compare(1) - case HashRequirement(otherPartitioning, _) => + case HashShuffleSpec(otherPartitioning, _) => if (partitioning.numPartitions != otherPartitioning.numPartitions) { partitioning.numPartitions.compare(otherPartitioning.numPartitions) } else { partitioning.expressions.length.compare(otherPartitioning.expressions.length) } - case RequirementCollection(requirements) => + case ShuffleSpecCollection(requirements) => // pick the best requirement in the other collection requirements.map(compare).min } @@ -475,7 +482,7 @@ case class HashRequirement( } } -case class RequirementCollection(requirements: Seq[Requirement]) extends Requirement { +case class ShuffleSpecCollection(requirements: Seq[ShuffleSpec]) extends ShuffleSpec { override def isCompatibleWith( otherPartitioning: Partitioning, otherClustering: Seq[Expression]): Boolean = { @@ -488,7 +495,7 @@ case class RequirementCollection(requirements: Seq[Requirement]) extends Require requirements.max.createPartitioning(clustering) } - override def compare(that: Requirement): Int = { + override def compare(that: ShuffleSpec): Int = { requirements.map(_.compare(that)).max } } From b184af9ad17f43c1312807e74a7130d16af11726 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 17 Sep 2021 09:41:27 -0700 Subject: [PATCH 05/29] new changes --- .../plans/physical/partitioning.scala | 154 +++++++++--------- .../sql/execution/adaptive/AQEUtils.scala | 8 +- .../exchange/EnsureRequirements.scala | 46 +++--- .../exchange/ValidateRequirements.scala | 1 - 4 files changed, 108 insertions(+), 101 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index efcbfb23228b0..9bc09187ca966 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -149,13 +149,20 @@ trait Partitioning { } /** - * Only return non-empty if the requirement can be used to repartition another side to match + * Creates a shuffle spec for this partitioning and its required distribution. The + * spec is used in the scenario where an operator has multiple children (e.g., join), and is + * used to decide whether this child is co-partitioned with others, therefore whether extra + * shuffle shall be introduced. + * + * Only return non-empty if the spec can be used to repartition another side to match * the distribution of this side. */ - final def createRequirement(distribution: Distribution): Option[ShuffleSpec] = + final def createShuffleSpec( + defaultNumPartitions: Int, + distribution: Distribution): Option[ShuffleSpec] = distribution match { case clustered: ClusteredDistribution => - createRequirement0(clustered) + createShuffleSpec0(defaultNumPartitions, clustered) case _ => throw new IllegalStateException(s"Unexpected distribution: " + s"${distribution.getClass.getSimpleName}") @@ -175,8 +182,9 @@ trait Partitioning { case _ => false } - protected def createRequirement0(distribution: ClusteredDistribution): Option[ShuffleSpec] = - None + protected def createShuffleSpec0( + defaultNumPartitions: Int, + distribution: ClusteredDistribution): Option[ShuffleSpec] = None } case class UnknownPartitioning(numPartitions: Int) extends Partitioning @@ -196,8 +204,11 @@ case object SinglePartition extends Partitioning { case _ => true } - override protected def createRequirement0( - distribution: ClusteredDistribution): Option[ShuffleSpec] = Some(SinglePartitionShuffleSpec$) + override protected def createShuffleSpec0( + defaultNumPartitions: Int, + distribution: ClusteredDistribution): Option[ShuffleSpec] = { + Some(SinglePartitionShuffleSpec) + } } /** @@ -222,7 +233,9 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } - override def createRequirement0(distribution: ClusteredDistribution): Option[ShuffleSpec] = { + override def createShuffleSpec0( + defaultNumPartitions: Int, + distribution: ClusteredDistribution): Option[ShuffleSpec] = { Some(HashShuffleSpec(this, distribution)) } @@ -284,6 +297,13 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) } } + override def createShuffleSpec0( + defaultNumPartitions: Int, + distribution: ClusteredDistribution): Option[ShuffleSpec] = { + Some(RangeShuffleSpec( + distribution.requiredNumPartitions.getOrElse(defaultNumPartitions), distribution)) + } + override protected def withNewChildrenInternal( newChildren: IndexedSeq[Expression]): RangePartitioning = copy(ordering = newChildren.asInstanceOf[Seq[SortOrder]]) @@ -326,10 +346,14 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def satisfies0(required: Distribution): Boolean = partitionings.exists(_.satisfies(required)) - override def createRequirement0(distribution: ClusteredDistribution): Option[ShuffleSpec] = { + override def createShuffleSpec0( + defaultNumPartitions: Int, + distribution: ClusteredDistribution): Option[ShuffleSpec] = { + require(satisfies(distribution), "createShuffleSpec should only be called after satisfies " + + "check is successful.") val eligible = partitionings .filter(_.satisfies(distribution)) - .flatMap(_.createRequirement(distribution)) + .flatMap(_.createShuffleSpec(defaultNumPartitions, distribution)) if (eligible.nonEmpty) { Some(ShuffleSpecCollection(eligible)) } else { @@ -361,22 +385,22 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { } /** - * This specifies that, when a operator has more than one children where each of which has - * its own partitioning and required distribution, the requirement for the other children to be - * co-partitioned with the current child. - */ - -/** - * This is used in the scenario where an operator has multiple children (e.g., join), and each of - * which has its own partitioning and required distribution. The spec is mainly used for two things: + * This is used in the scenario where an operator has multiple children (e.g., join) and one or more + * of which have their own requirement regarding whether its data can be considered as + * co-partitioned from others. This offers APIs for: * - * 1. Compare with specs from other children and check if they are compatible. When two specs - * are compatible, we can say their data are co-partitioned, and thus will allow Spark to - * eliminate shuffle in operators such as join. - * 2. In case this spec is not compatible with another, create a partitioning that can be used to - * re-partition the other side. + * 1. Comparing with specs from other children of the operator and check if they are compatible. + * When two specs are compatible, we can say their data are co-partitioned, and Spark will + * potentially able to eliminate shuffle if necessary. + * 2. Creating a partitioning that can be used to re-partition another child, so that to make it + * having a compatible partitioning as this node. */ -trait ShuffleSpec extends Ordered[ShuffleSpec] { +trait ShuffleSpec { + /** + * Returns the number of partitions of this shuffle spec + */ + def numPartitions: Int + /** * Returns true iff this spec is compatible with the other [[Partitioning]] and * clustering expressions (e.g., from [[ClusteredDistribution]]). @@ -388,7 +412,7 @@ trait ShuffleSpec extends Ordered[ShuffleSpec] { def isCompatibleWith(other: ShuffleSpec): Boolean /** - * Create a partitioning that can be used to re-partitioned the other side whose required + * Creates a partitioning that can be used to re-partitioned the other side whose required * distribution is specified via `clustering`. * * Note: this will only be called after `isCompatibleWith` returns true on the side where the @@ -397,23 +421,25 @@ trait ShuffleSpec extends Ordered[ShuffleSpec] { def createPartitioning(clustering: Seq[Expression]): Partitioning } -case object SinglePartitionShuffleSpec$ extends ShuffleSpec { - override def isCompatibleWith( - otherPartitioning: Partitioning, - otherClustering: Seq[Expression]): Boolean = { - otherPartitioning.numPartitions == 1 +case object SinglePartitionShuffleSpec extends ShuffleSpec { + override def isCompatibleWith(other: ShuffleSpec): Boolean = { + other.numPartitions == numPartitions } override def createPartitioning(clustering: Seq[Expression]): Partitioning = SinglePartition - override def compare(that: ShuffleSpec): Int = that match { - case SinglePartitionShuffleSpec$ => - 0 - case HashShuffleSpec(partitioning, _) => - 1.compare(partitioning.numPartitions) - case ShuffleSpecCollection(requirements) => - requirements.map(compare).min + override def numPartitions: Int = 1 +} + +case class RangeShuffleSpec( + numPartitions: Int, + distribution: ClusteredDistribution) extends ShuffleSpec { + + override def isCompatibleWith(other: ShuffleSpec): Boolean = false + + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { + distribution.createPartitioning(numPartitions) } } @@ -422,20 +448,19 @@ case class HashShuffleSpec( distribution: ClusteredDistribution) extends ShuffleSpec { private lazy val matchingIndexes = indexMap(distribution.clustering, partitioning.expressions) - override def isCompatibleWith( - otherPartitioning: Partitioning, - otherClustering: Seq[Expression]): Boolean = otherPartitioning match { - case SinglePartition => + override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { + case SinglePartitionShuffleSpec => partitioning.numPartitions == 1 - case HashPartitioning(expressions, _) => + case HashShuffleSpec(partitioning, distribution) => + val expressions = partitioning.expressions // we need to check: // 1. both partitioning have the same number of expressions // 2. each corresponding expression in both partitioning is used in the same positions // of the corresponding distribution. partitioning.expressions.length == expressions.length && - matchingIndexes == indexMap(otherClustering, expressions) - case PartitioningCollection(partitionings) => - partitionings.exists(isCompatibleWith(_, otherClustering)) + matchingIndexes == indexMap(distribution.clustering, expressions) + case ShuffleSpecCollection(specs) => + specs.exists(isCompatibleWith) case _ => false } @@ -448,25 +473,13 @@ case class HashShuffleSpec( HashPartitioning(exprs, partitioning.numPartitions) } - override def compare(that: ShuffleSpec): Int = that match { - case SinglePartitionShuffleSpec$ => - partitioning.numPartitions.compare(1) - case HashShuffleSpec(otherPartitioning, _) => - if (partitioning.numPartitions != otherPartitioning.numPartitions) { - partitioning.numPartitions.compare(otherPartitioning.numPartitions) - } else { - partitioning.expressions.length.compare(otherPartitioning.expressions.length) - } - case ShuffleSpecCollection(requirements) => - // pick the best requirement in the other collection - requirements.map(compare).min - } + override def numPartitions: Int = partitioning.numPartitions - // For each expression in the `HashPartitioning` that has occurrences in - // `ClusteredDistribution`, returns a mapping from its index in the partitioning to the + // For each expression in the `expressions` that has occurrences in + // `clustering`, returns a mapping from its index in the partitioning to the // indexes where it appears in the distribution. - // For instance, if `partitioning` is `[a, b]` and `distribution is `[a, a, b]`, then the - // result mapping could be `{ 0 -> (0, 1), 1 -> (2) }`. + // For instance, if `clustering` is [a, b] and `expressions` is [a, a, b], then the + // result mapping could be { 0 -> (0, 1), 1 -> (2) }. private def indexMap( clustering: Seq[Expression], expressions: Seq[Expression]): mutable.Map[Int, mutable.BitSet] = { @@ -482,20 +495,15 @@ case class HashShuffleSpec( } } -case class ShuffleSpecCollection(requirements: Seq[ShuffleSpec]) extends ShuffleSpec { - override def isCompatibleWith( - otherPartitioning: Partitioning, - otherClustering: Seq[Expression]): Boolean = { - requirements.exists(_.isCompatibleWith(otherPartitioning, otherClustering)) +case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec { + override def isCompatibleWith(other: ShuffleSpec): Boolean = { + specs.exists(_.isCompatibleWith(this)) } override def createPartitioning(clustering: Seq[Expression]): Partitioning = { - // choose the best requirement (e.g., maximum shuffle parallelism, min shuffle data size, etc) - // from the collection and use that to repartition the other sides - requirements.max.createPartitioning(clustering) + // choose the spec with the max number of partitions for better parallelism + specs.maxBy(_.numPartitions).createPartitioning(clustering) } - override def compare(that: ShuffleSpec): Int = { - requirements.map(_.compare(that)).max - } + override def numPartitions: Int = specs.maxBy(_.numPartitions).numPartitions } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala index 277af212d81f3..cbd4ee698df28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.adaptive -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution, HashPartitioning, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, HashPartitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{CollectMetricsExec, FilterExec, ProjectExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.exchange.{REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeExec} @@ -37,14 +37,14 @@ object AQEUtils { } else { None } - Some(HashClusteredDistribution(h.expressions, numPartitions)) + Some(ClusteredDistribution(h.expressions, numPartitions)) case f: FilterExec => getRequiredDistribution(f.child) case s: SortExec if !s.global => getRequiredDistribution(s.child) case c: CollectMetricsExec => getRequiredDistribution(c.child) case p: ProjectExec => getRequiredDistribution(p.child).flatMap { - case h: HashClusteredDistribution => - if (h.expressions.forall(e => p.projectList.exists(_.semanticEquals(e)))) { + case h: ClusteredDistribution => + if (h.clustering.forall(e => p.projectList.exists(_.semanticEquals(e)))) { Some(h) } else { // It's possible that the user-specified repartition is effective but the output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 04d171f484235..9402b2e1fcd67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -78,37 +78,37 @@ case class EnsureRequirements( if (childrenIndexes.length > 1 && childrenIndexes.map(requiredChildDistributions(_)) .forall(_.isInstanceOf[ClusteredDistribution])) { // TODO: can we handle AllTuples together? - val childrenWithDistribution = childrenIndexes - .map(i => (children(i).outputPartitioning, - requiredChildDistributions(i).asInstanceOf[ClusteredDistribution])) - val requirements = childrenWithDistribution.flatMap(pd => pd._1.createRequirement(pd._2)) + val specs = childrenIndexes.map(i => + i -> newChildren(i).outputPartitioning.createShuffleSpec( + requiredChildDistributions(i).asInstanceOf[ClusteredDistribution]) + ).toMap - children = children.zip(requiredChildDistributions).zipWithIndex.map { - case ((child, _), idx) if !childrenIndexes.contains(idx) => - child - case ((child, dist), _) => - val partitioningOpt = if (requirements.isEmpty) { - // all the children need to be re-shuffled - val numPartitions = dist.requiredNumPartitions.getOrElse(conf.numShufflePartitions) - Some(dist.createPartitioning(numPartitions)) - } else { + if (specs.values.exists(_.isDefined)) { + // one or more children have requirement on others for shuffle, so we need to check + // compatibility and come up a common partitioning for them + val bestSpec = specs.values.flatten.maxBy(_.numPartitions) + + newChildren = newChildren.zip(requiredChildDistributions).zipWithIndex.map { + case ((child, _), idx) if !childrenIndexes.contains(idx) => + child + case ((child, dist), idx) => // pick the best candidate from the requirements and use that to re-shuffle other // children if necessary - val best = requirements.max val clustering = dist.asInstanceOf[ClusteredDistribution].clustering - if (best.isCompatibleWith(child.outputPartitioning, clustering)) { + val specOpt = specs(idx) + val partitioningOpt = if (specOpt.isDefined && bestSpec.isCompatibleWith(specOpt.get)) { None } else { - Some(best.createPartitioning(clustering)) + Some(bestSpec.createPartitioning(clustering)) } - } - partitioningOpt.map { p => - child match { - case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(p, c, so) - case _ => ShuffleExchangeExec(p, child) - } - }.getOrElse(child) + partitioningOpt.map { p => + child match { + case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(p, c, so) + case _ => ShuffleExchangeExec(p, child) + } + }.getOrElse(child) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ValidateRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ValidateRequirements.scala index 5003db6a16753..9538199590477 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ValidateRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ValidateRequirements.scala @@ -49,7 +49,6 @@ object ValidateRequirements extends Logging { // have the same number of partitions. val numPartitions = requiredChildDistributions.zipWithIndex.collect { case (_: ClusteredDistribution, i) => i - case (_: HashClusteredDistribution, i) => i }.map(i => children(i).outputPartitioning.numPartitions) if (numPartitions.length > 1 && !numPartitions.tail.forall(_ == numPartitions.head)) { logDebug(s"ValidateRequirements failed: different partition num in\n$plan") From 3efc11d744af529e7b83c344bcffac651a88cdc2 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 17 Sep 2021 09:47:26 -0700 Subject: [PATCH 06/29] remove option from the interface --- .../plans/physical/partitioning.scala | 30 +++++------- .../exchange/EnsureRequirements.scala | 46 +++++++++---------- 2 files changed, 34 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 9bc09187ca966..45d58eac0ab48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -159,7 +159,7 @@ trait Partitioning { */ final def createShuffleSpec( defaultNumPartitions: Int, - distribution: Distribution): Option[ShuffleSpec] = + distribution: Distribution): ShuffleSpec = distribution match { case clustered: ClusteredDistribution => createShuffleSpec0(defaultNumPartitions, clustered) @@ -184,7 +184,9 @@ trait Partitioning { protected def createShuffleSpec0( defaultNumPartitions: Int, - distribution: ClusteredDistribution): Option[ShuffleSpec] = None + distribution: ClusteredDistribution): ShuffleSpec = + throw new IllegalStateException("createShuffleSpec0 is called on unexpected " + + s"partitioning: $this") } case class UnknownPartitioning(numPartitions: Int) extends Partitioning @@ -206,9 +208,7 @@ case object SinglePartition extends Partitioning { override protected def createShuffleSpec0( defaultNumPartitions: Int, - distribution: ClusteredDistribution): Option[ShuffleSpec] = { - Some(SinglePartitionShuffleSpec) - } + distribution: ClusteredDistribution): ShuffleSpec = SinglePartitionShuffleSpec } /** @@ -235,9 +235,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) override def createShuffleSpec0( defaultNumPartitions: Int, - distribution: ClusteredDistribution): Option[ShuffleSpec] = { - Some(HashShuffleSpec(this, distribution)) - } + distribution: ClusteredDistribution): ShuffleSpec = HashShuffleSpec(this, distribution) /** * Returns an expression that will produce a valid partition ID(i.e. non-negative and is less @@ -299,9 +297,9 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) override def createShuffleSpec0( defaultNumPartitions: Int, - distribution: ClusteredDistribution): Option[ShuffleSpec] = { - Some(RangeShuffleSpec( - distribution.requiredNumPartitions.getOrElse(defaultNumPartitions), distribution)) + distribution: ClusteredDistribution): ShuffleSpec = { + RangeShuffleSpec( + distribution.requiredNumPartitions.getOrElse(defaultNumPartitions), distribution) } override protected def withNewChildrenInternal( @@ -348,17 +346,13 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def createShuffleSpec0( defaultNumPartitions: Int, - distribution: ClusteredDistribution): Option[ShuffleSpec] = { + distribution: ClusteredDistribution): ShuffleSpec = { require(satisfies(distribution), "createShuffleSpec should only be called after satisfies " + "check is successful.") val eligible = partitionings .filter(_.satisfies(distribution)) - .flatMap(_.createShuffleSpec(defaultNumPartitions, distribution)) - if (eligible.nonEmpty) { - Some(ShuffleSpecCollection(eligible)) - } else { - None - } + .map(_.createShuffleSpec(defaultNumPartitions, distribution)) + ShuffleSpecCollection(eligible) } override def toString: String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 9402b2e1fcd67..7be4d2c40e9ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -80,35 +80,33 @@ case class EnsureRequirements( // TODO: can we handle AllTuples together? val specs = childrenIndexes.map(i => i -> newChildren(i).outputPartitioning.createShuffleSpec( + conf.numShufflePartitions, requiredChildDistributions(i).asInstanceOf[ClusteredDistribution]) ).toMap - if (specs.values.exists(_.isDefined)) { - // one or more children have requirement on others for shuffle, so we need to check - // compatibility and come up a common partitioning for them - val bestSpec = specs.values.flatten.maxBy(_.numPartitions) + // one or more children have requirement on others for shuffle, so we need to check + // compatibility and come up a common partitioning for them + val bestSpec = specs.values.maxBy(_.numPartitions) - newChildren = newChildren.zip(requiredChildDistributions).zipWithIndex.map { - case ((child, _), idx) if !childrenIndexes.contains(idx) => - child - case ((child, dist), idx) => - // pick the best candidate from the requirements and use that to re-shuffle other - // children if necessary - val clustering = dist.asInstanceOf[ClusteredDistribution].clustering - val specOpt = specs(idx) - val partitioningOpt = if (specOpt.isDefined && bestSpec.isCompatibleWith(specOpt.get)) { - None - } else { - Some(bestSpec.createPartitioning(clustering)) - } + newChildren = newChildren.zip(requiredChildDistributions).zipWithIndex.map { + case ((child, _), idx) if !childrenIndexes.contains(idx) => + child + case ((child, dist), idx) => + // pick the best candidate from the requirements and use that to re-shuffle other + // children if necessary + val clustering = dist.asInstanceOf[ClusteredDistribution].clustering + val partitioningOpt = if (bestSpec.isCompatibleWith(specs(idx))) { + None + } else { + Some(bestSpec.createPartitioning(clustering)) + } - partitioningOpt.map { p => - child match { - case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(p, c, so) - case _ => ShuffleExchangeExec(p, child) - } - }.getOrElse(child) - } + partitioningOpt.map { p => + child match { + case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(p, c, so) + case _ => ShuffleExchangeExec(p, child) + } + }.getOrElse(child) } } From 5e4fca5f43db7d528d0f0a653b6d791c8a1dba25 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 17 Sep 2021 12:40:54 -0700 Subject: [PATCH 07/29] more --- .../plans/physical/partitioning.scala | 40 ++++++++++++------- .../sql/catalyst/DistributionSuite.scala | 16 +++----- .../exchange/EnsureRequirements.scala | 26 +++++------- 3 files changed, 42 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 45d58eac0ab48..f56bbb27472de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -154,13 +154,16 @@ trait Partitioning { * used to decide whether this child is co-partitioned with others, therefore whether extra * shuffle shall be introduced. * - * Only return non-empty if the spec can be used to repartition another side to match - * the distribution of this side. + * @param defaultNumPartitions the default number of partitions to use when creating a new + * partitioning from the spec + * @param distribution the required distribution for this partitioning */ final def createShuffleSpec( defaultNumPartitions: Int, distribution: Distribution): ShuffleSpec = distribution match { + case AllTuples => + SinglePartitionShuffleSpec case clustered: ClusteredDistribution => createShuffleSpec0(defaultNumPartitions, clustered) case _ => @@ -185,8 +188,7 @@ trait Partitioning { protected def createShuffleSpec0( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = - throw new IllegalStateException("createShuffleSpec0 is called on unexpected " + - s"partitioning: $this") + throw new IllegalStateException(s"Unexpected partitioning: ${getClass.getSimpleName}") } case class UnknownPartitioning(numPartitions: Int) extends Partitioning @@ -298,8 +300,8 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) override def createShuffleSpec0( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = { - RangeShuffleSpec( - distribution.requiredNumPartitions.getOrElse(defaultNumPartitions), distribution) + val numPartitions = distribution.requiredNumPartitions.getOrElse(defaultNumPartitions) + RangeShuffleSpec(numPartitions, distribution) } override protected def withNewChildrenInternal( @@ -406,13 +408,23 @@ trait ShuffleSpec { def isCompatibleWith(other: ShuffleSpec): Boolean /** - * Creates a partitioning that can be used to re-partitioned the other side whose required - * distribution is specified via `clustering`. + * Creates a partitioning that can be used to re-partitioned the other side with the given + * required distribution. * * Note: this will only be called after `isCompatibleWith` returns true on the side where the * `clustering` is returned from. */ - def createPartitioning(clustering: Seq[Expression]): Partitioning + final def createPartitioning(distribution: Distribution): Partitioning = distribution match { + case AllTuples => + SinglePartition + case ClusteredDistribution(clustering, _) => + createPartitioning0(clustering) + case _ => + throw new IllegalStateException("unexpected distribution: " + + s"${distribution.getClass.getSimpleName}") + } + + def createPartitioning0(clustering: Seq[Expression]): Partitioning } case object SinglePartitionShuffleSpec extends ShuffleSpec { @@ -420,7 +432,7 @@ case object SinglePartitionShuffleSpec extends ShuffleSpec { other.numPartitions == numPartitions } - override def createPartitioning(clustering: Seq[Expression]): Partitioning = + override def createPartitioning0(clustering: Seq[Expression]): Partitioning = SinglePartition override def numPartitions: Int = 1 @@ -432,7 +444,7 @@ case class RangeShuffleSpec( override def isCompatibleWith(other: ShuffleSpec): Boolean = false - override def createPartitioning(clustering: Seq[Expression]): Partitioning = { + override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { distribution.createPartitioning(numPartitions) } } @@ -459,7 +471,7 @@ case class HashShuffleSpec( false } - override def createPartitioning(clustering: Seq[Expression]): Partitioning = { + override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { val exprs = clustering .zipWithIndex .filter(x => matchingIndexes.keySet.contains(x._2)) @@ -494,9 +506,9 @@ case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec { specs.exists(_.isCompatibleWith(this)) } - override def createPartitioning(clustering: Seq[Expression]): Partitioning = { + override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { // choose the spec with the max number of partitions for better parallelism - specs.maxBy(_.numPartitions).createPartitioning(clustering) + specs.maxBy(_.numPartitions).createPartitioning0(clustering) } override def numPartitions: Int = specs.maxBy(_.numPartitions).numPartitions diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index a77560cf3faac..5d3f960c3bfac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -31,16 +31,12 @@ class DistributionSuite extends SparkFunSuite { if (inputPartitioning.satisfies(requiredDistribution) != satisfied) { fail( s""" - |== Input Partitioning == - |$inputPartitioning - - |== Required Distri - - |$requiredDistribution - - |== Does input partitioning satisfy require - ution? == - |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} + |== Input Partitioning == + |$inputPartitioning + |== Required Distribution == + |$requiredDistribution + |== Does input partitioning satisfy required distribution? == + |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 7be4d2c40e9ce..8760e44a376a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -75,38 +75,32 @@ case class EnsureRequirements( // If there are more than one children, we'll need to check partitioning & distribution of them // and see if extra shuffles are necessary. - if (childrenIndexes.length > 1 && childrenIndexes.map(requiredChildDistributions(_)) - .forall(_.isInstanceOf[ClusteredDistribution])) { - // TODO: can we handle AllTuples together? + if (childrenIndexes.length > 1) { val specs = childrenIndexes.map(i => - i -> newChildren(i).outputPartitioning.createShuffleSpec( + i -> children(i).outputPartitioning.createShuffleSpec( conf.numShufflePartitions, - requiredChildDistributions(i).asInstanceOf[ClusteredDistribution]) + requiredChildDistributions(i)) ).toMap // one or more children have requirement on others for shuffle, so we need to check // compatibility and come up a common partitioning for them val bestSpec = specs.values.maxBy(_.numPartitions) - newChildren = newChildren.zip(requiredChildDistributions).zipWithIndex.map { + children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, _), idx) if !childrenIndexes.contains(idx) => child case ((child, dist), idx) => // pick the best candidate from the requirements and use that to re-shuffle other // children if necessary - val clustering = dist.asInstanceOf[ClusteredDistribution].clustering - val partitioningOpt = if (bestSpec.isCompatibleWith(specs(idx))) { - None + if (bestSpec.isCompatibleWith(specs(idx))) { + child } else { - Some(bestSpec.createPartitioning(clustering)) - } - - partitioningOpt.map { p => + val newPartitioning = bestSpec.createPartitioning(dist) child match { - case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(p, c, so) - case _ => ShuffleExchangeExec(p, child) + case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(newPartitioning, c, so) + case _ => ShuffleExchangeExec(newPartitioning, child) } - }.getOrElse(child) + } } } From 1a8301e72e4e071df6bb4df6dbad72d1d70b778e Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 17 Sep 2021 12:48:50 -0700 Subject: [PATCH 08/29] update manifest files --- .../approved-plans-v1_4/q17.sf100/explain.txt | 341 +++++++------- .../q17.sf100/simplified.txt | 155 +++---- .../approved-plans-v1_4/q25.sf100/explain.txt | 343 +++++++------- .../q25.sf100/simplified.txt | 155 +++---- .../approved-plans-v1_4/q29.sf100/explain.txt | 376 ++++++++-------- .../q29.sf100/simplified.txt | 157 +++---- .../approved-plans-v1_4/q47.sf100/explain.txt | 16 + .../approved-plans-v1_4/q57.sf100/explain.txt | 16 + .../approved-plans-v1_4/q72.sf100/explain.txt | 426 ++++++++++-------- .../q72.sf100/simplified.txt | 216 +++++---- .../approved-plans-v2_7/q24.sf100/explain.txt | 104 +++-- .../q24.sf100/simplified.txt | 69 +-- .../approved-plans-v2_7/q47.sf100/explain.txt | 16 + .../q51a.sf100/explain.txt | 16 + .../approved-plans-v2_7/q57.sf100/explain.txt | 16 + .../approved-plans-v2_7/q72.sf100/explain.txt | 426 ++++++++++-------- .../q72.sf100/simplified.txt | 216 +++++---- 17 files changed, 1611 insertions(+), 1453 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index d61798f6ad06e..91ed657d870d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -1,53 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * SortMergeJoin Inner (44) - :- * Sort (35) - : +- Exchange (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.store (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (31) - : +- Exchange (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.store_returns (24) - : +- ReusedExchange (27) - +- * Sort (43) - +- Exchange (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.catalog_sales (36) - +- ReusedExchange (39) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Project (32) + : +- * SortMergeJoin Inner (31) + : :- * Sort (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.store (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (30) + : +- Exchange (29) + : +- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.store_returns (23) + : +- ReusedExchange (26) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet default.catalog_sales (33) + +- ReusedExchange (36) (1) Scan parquet default.store_sales @@ -65,7 +62,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#8] (5) BroadcastHashJoin [codegen id : 3] @@ -143,194 +140,200 @@ Join condition: None Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_sk#13, i_item_id#14, i_item_desc#15] -(22) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [id=#17] - -(23) Sort [codegen id : 8] +(22) Sort [codegen id : 7] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15] Arguments: [ss_customer_sk#2 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.store_returns -Output [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22] +(23) Scan parquet default.store_returns +Output [5]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#22), dynamicpruningexpression(sr_returned_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(sr_returned_date_sk#21), dynamicpruningexpression(sr_returned_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 10] -Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22] +(24) ColumnarToRow [codegen id : 9] +Input [5]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21] -(26) Filter [codegen id : 10] -Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22] -Condition : ((isnotnull(sr_customer_sk#19) AND isnotnull(sr_item_sk#18)) AND isnotnull(sr_ticket_number#20)) +(25) Filter [codegen id : 9] +Input [5]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21] +Condition : ((isnotnull(sr_customer_sk#18) AND isnotnull(sr_item_sk#17)) AND isnotnull(sr_ticket_number#19)) -(27) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#24] +(26) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#23] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [sr_returned_date_sk#22] -Right keys [1]: [d_date_sk#24] +(27) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [sr_returned_date_sk#21] +Right keys [1]: [d_date_sk#23] Join condition: None -(29) Project [codegen id : 10] -Output [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] -Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22, d_date_sk#24] +(28) Project [codegen id : 9] +Output [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20] +Input [6]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20, sr_returned_date_sk#21, d_date_sk#23] -(30) Exchange -Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] -Arguments: hashpartitioning(sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20, 5), ENSURE_REQUIREMENTS, [id=#25] +(29) Exchange +Input [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20] +Arguments: hashpartitioning(sr_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(31) Sort [codegen id : 11] -Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] -Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST, sr_ticket_number#20 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 10] +Input [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20] +Arguments: [sr_customer_sk#18 ASC NULLS FIRST, sr_item_sk#17 ASC NULLS FIRST, sr_ticket_number#19 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 12] +(31) SortMergeJoin [codegen id : 11] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20] +Right keys [3]: [sr_customer_sk#18, sr_item_sk#17, sr_ticket_number#19] Join condition: None -(33) Project [codegen id : 12] -Output [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] - -(34) Exchange -Input [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21] -Arguments: hashpartitioning(sr_customer_sk#19, sr_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26] - -(35) Sort [codegen id : 13] -Input [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21] -Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST], false, 0 +(32) Project [codegen id : 11] +Output [7]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#17, sr_customer_sk#18, sr_return_quantity#20] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20] -(36) Scan parquet default.catalog_sales -Output [4]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30] +(33) Scan parquet default.catalog_sales +Output [4]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#30), dynamicpruningexpression(cs_sold_date_sk#30 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(cs_sold_date_sk#28), dynamicpruningexpression(cs_sold_date_sk#28 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 15] -Input [4]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30] +(34) ColumnarToRow [codegen id : 13] +Input [4]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28] -(38) Filter [codegen id : 15] -Input [4]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30] -Condition : (isnotnull(cs_bill_customer_sk#27) AND isnotnull(cs_item_sk#28)) +(35) Filter [codegen id : 13] +Input [4]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28] +Condition : (isnotnull(cs_bill_customer_sk#25) AND isnotnull(cs_item_sk#26)) -(39) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#31] +(36) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#29] -(40) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#30] -Right keys [1]: [d_date_sk#31] +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#28] +Right keys [1]: [d_date_sk#29] Join condition: None -(41) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29] -Input [5]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29, cs_sold_date_sk#30, d_date_sk#31] +(38) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27] +Input [5]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_sk#28, d_date_sk#29] -(42) Exchange -Input [3]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29] -Arguments: hashpartitioning(cs_bill_customer_sk#27, cs_item_sk#28, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27] +Arguments: hashpartitioning(cs_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] -(43) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29] -Arguments: [cs_bill_customer_sk#27 ASC NULLS FIRST, cs_item_sk#28 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27] +Arguments: [cs_bill_customer_sk#25 ASC NULLS FIRST, cs_item_sk#26 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#19, sr_item_sk#18] -Right keys [2]: [cs_bill_customer_sk#27, cs_item_sk#28] +(41) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#18, sr_item_sk#17] +Right keys [2]: [cs_bill_customer_sk#25, cs_item_sk#26] Join condition: None -(45) Project [codegen id : 17] -Output [6]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#29, s_state#10, i_item_id#14, i_item_desc#15] -Input [10]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21, cs_bill_customer_sk#27, cs_item_sk#28, cs_quantity#29] +(42) Project [codegen id : 15] +Output [6]: [ss_quantity#5, sr_return_quantity#20, cs_quantity#27, s_state#10, i_item_id#14, i_item_desc#15] +Input [10]: [ss_quantity#5, s_state#10, i_item_id#14, i_item_desc#15, sr_item_sk#17, sr_customer_sk#18, sr_return_quantity#20, cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27] -(46) HashAggregate [codegen id : 17] -Input [6]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#29, s_state#10, i_item_id#14, i_item_desc#15] +(43) HashAggregate [codegen id : 15] +Input [6]: [ss_quantity#5, sr_return_quantity#20, cs_quantity#27, s_state#10, i_item_id#14, i_item_desc#15] Keys [3]: [i_item_id#14, i_item_desc#15, s_state#10] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#21), partial_avg(sr_return_quantity#21), partial_stddev_samp(cast(sr_return_quantity#21 as double)), partial_count(cs_quantity#29), partial_avg(cs_quantity#29), partial_stddev_samp(cast(cs_quantity#29 as double))] -Aggregate Attributes [18]: [count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44, count#45, sum#46, count#47, n#48, avg#49, m2#50] -Results [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62, count#63, sum#64, count#65, n#66, avg#67, m2#68] +Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#20), partial_avg(sr_return_quantity#20), partial_stddev_samp(cast(sr_return_quantity#20 as double)), partial_count(cs_quantity#27), partial_avg(cs_quantity#27), partial_stddev_samp(cast(cs_quantity#27 as double))] +Aggregate Attributes [18]: [count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] +Results [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#49, sum#50, count#51, n#52, avg#53, m2#54, count#55, sum#56, count#57, n#58, avg#59, m2#60, count#61, sum#62, count#63, n#64, avg#65, m2#66] -(47) Exchange -Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62, count#63, sum#64, count#65, n#66, avg#67, m2#68] -Arguments: hashpartitioning(i_item_id#14, i_item_desc#15, s_state#10, 5), ENSURE_REQUIREMENTS, [id=#69] +(44) Exchange +Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#49, sum#50, count#51, n#52, avg#53, m2#54, count#55, sum#56, count#57, n#58, avg#59, m2#60, count#61, sum#62, count#63, n#64, avg#65, m2#66] +Arguments: hashpartitioning(i_item_id#14, i_item_desc#15, s_state#10, 5), ENSURE_REQUIREMENTS, [id=#67] -(48) HashAggregate [codegen id : 18] -Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62, count#63, sum#64, count#65, n#66, avg#67, m2#68] +(45) HashAggregate [codegen id : 16] +Input [21]: [i_item_id#14, i_item_desc#15, s_state#10, count#49, sum#50, count#51, n#52, avg#53, m2#54, count#55, sum#56, count#57, n#58, avg#59, m2#60, count#61, sum#62, count#63, n#64, avg#65, m2#66] Keys [3]: [i_item_id#14, i_item_desc#15, s_state#10] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#21), avg(sr_return_quantity#21), stddev_samp(cast(sr_return_quantity#21 as double)), count(cs_quantity#29), avg(cs_quantity#29), stddev_samp(cast(cs_quantity#29 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#70, avg(ss_quantity#5)#71, stddev_samp(cast(ss_quantity#5 as double))#72, count(sr_return_quantity#21)#73, avg(sr_return_quantity#21)#74, stddev_samp(cast(sr_return_quantity#21 as double))#75, count(cs_quantity#29)#76, avg(cs_quantity#29)#77, stddev_samp(cast(cs_quantity#29 as double))#78] -Results [15]: [i_item_id#14, i_item_desc#15, s_state#10, count(ss_quantity#5)#70 AS store_sales_quantitycount#79, avg(ss_quantity#5)#71 AS store_sales_quantityave#80, stddev_samp(cast(ss_quantity#5 as double))#72 AS store_sales_quantitystdev#81, (stddev_samp(cast(ss_quantity#5 as double))#72 / avg(ss_quantity#5)#71) AS store_sales_quantitycov#82, count(sr_return_quantity#21)#73 AS as_store_returns_quantitycount#83, avg(sr_return_quantity#21)#74 AS as_store_returns_quantityave#84, stddev_samp(cast(sr_return_quantity#21 as double))#75 AS as_store_returns_quantitystdev#85, (stddev_samp(cast(sr_return_quantity#21 as double))#75 / avg(sr_return_quantity#21)#74) AS store_returns_quantitycov#86, count(cs_quantity#29)#76 AS catalog_sales_quantitycount#87, avg(cs_quantity#29)#77 AS catalog_sales_quantityave#88, (stddev_samp(cast(cs_quantity#29 as double))#78 / avg(cs_quantity#29)#77) AS catalog_sales_quantitystdev#89, (stddev_samp(cast(cs_quantity#29 as double))#78 / avg(cs_quantity#29)#77) AS catalog_sales_quantitycov#90] +Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#20), avg(sr_return_quantity#20), stddev_samp(cast(sr_return_quantity#20 as double)), count(cs_quantity#27), avg(cs_quantity#27), stddev_samp(cast(cs_quantity#27 as double))] +Aggregate Attributes [9]: [count(ss_quantity#5)#68, avg(ss_quantity#5)#69, stddev_samp(cast(ss_quantity#5 as double))#70, count(sr_return_quantity#20)#71, avg(sr_return_quantity#20)#72, stddev_samp(cast(sr_return_quantity#20 as double))#73, count(cs_quantity#27)#74, avg(cs_quantity#27)#75, stddev_samp(cast(cs_quantity#27 as double))#76] +Results [15]: [i_item_id#14, i_item_desc#15, s_state#10, count(ss_quantity#5)#68 AS store_sales_quantitycount#77, avg(ss_quantity#5)#69 AS store_sales_quantityave#78, stddev_samp(cast(ss_quantity#5 as double))#70 AS store_sales_quantitystdev#79, (stddev_samp(cast(ss_quantity#5 as double))#70 / avg(ss_quantity#5)#69) AS store_sales_quantitycov#80, count(sr_return_quantity#20)#71 AS as_store_returns_quantitycount#81, avg(sr_return_quantity#20)#72 AS as_store_returns_quantityave#82, stddev_samp(cast(sr_return_quantity#20 as double))#73 AS as_store_returns_quantitystdev#83, (stddev_samp(cast(sr_return_quantity#20 as double))#73 / avg(sr_return_quantity#20)#72) AS store_returns_quantitycov#84, count(cs_quantity#27)#74 AS catalog_sales_quantitycount#85, avg(cs_quantity#27)#75 AS catalog_sales_quantityave#86, (stddev_samp(cast(cs_quantity#27 as double))#76 / avg(cs_quantity#27)#75) AS catalog_sales_quantitystdev#87, (stddev_samp(cast(cs_quantity#27 as double))#76 / avg(cs_quantity#27)#75) AS catalog_sales_quantitycov#88] -(49) TakeOrderedAndProject -Input [15]: [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90] +(46) TakeOrderedAndProject +Input [15]: [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#77, store_sales_quantityave#78, store_sales_quantitystdev#79, store_sales_quantitycov#80, as_store_returns_quantitycount#81, as_store_returns_quantityave#82, as_store_returns_quantitystdev#83, store_returns_quantitycov#84, catalog_sales_quantitycount#85, catalog_sales_quantityave#86, catalog_sales_quantitystdev#87, catalog_sales_quantitycov#88] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#77, store_sales_quantityave#78, store_sales_quantitystdev#79, store_sales_quantitycov#80, as_store_returns_quantitycount#81, as_store_returns_quantityave#82, as_store_returns_quantitystdev#83, store_returns_quantitycov#84, catalog_sales_quantitycount#85, catalog_sales_quantityave#86, catalog_sales_quantitystdev#87, catalog_sales_quantitycov#88] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (54) -+- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) +BroadcastExchange (51) ++- * Project (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.date_dim (47) -(50) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_quarter_name#91] +(47) Scan parquet default.date_dim +Output [2]: [d_date_sk#8, d_quarter_name#89] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_quarter_name#91] +(48) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#8, d_quarter_name#89] -(52) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_quarter_name#91] -Condition : ((isnotnull(d_quarter_name#91) AND (d_quarter_name#91 = 2001Q1)) AND isnotnull(d_date_sk#8)) +(49) Filter [codegen id : 1] +Input [2]: [d_date_sk#8, d_quarter_name#89] +Condition : ((isnotnull(d_quarter_name#89) AND (d_quarter_name#89 = 2001Q1)) AND isnotnull(d_date_sk#8)) -(53) Project [codegen id : 1] +(50) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_quarter_name#91] +Input [2]: [d_date_sk#8, d_quarter_name#89] -(54) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] - -Subquery:2 Hosting operator id = 24 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet default.date_dim (55) - - -(55) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_quarter_name#93] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#92] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#21 IN dynamicpruning#22 +BroadcastExchange (56) ++- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet default.date_dim (52) + + +(52) Scan parquet default.date_dim +Output [2]: [d_date_sk#23, d_quarter_name#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_quarter_name#93] +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#23, d_quarter_name#91] -(57) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_quarter_name#93] -Condition : (d_quarter_name#93 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +(54) Filter [codegen id : 1] +Input [2]: [d_date_sk#23, d_quarter_name#91] +Condition : (d_quarter_name#91 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#23)) -(58) Project [codegen id : 1] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_quarter_name#93] +(55) Project [codegen id : 1] +Output [1]: [d_date_sk#23] +Input [2]: [d_date_sk#23, d_quarter_name#91] +<<<<<<< HEAD (59) BroadcastExchange Input [1]: [d_date_sk#24] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#94] +======= +(56) BroadcastExchange +Input [1]: [d_date_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) -Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#30 IN dynamicpruning#23 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#28 IN dynamicpruning#22 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index 06c8f7b3912e5..693bc90b720eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -1,97 +1,90 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (18) + WholeStageCodegen (16) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (17) + WholeStageCodegen (15) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] + WholeStageCodegen (11) + Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (7) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] + Scan parquet default.date_dim [d_date_sk,d_quarter_name] InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] - InputAdapter - WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 - WholeStageCodegen (10) - Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_state] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (10) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk] #6 + WholeStageCodegen (9) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_quarter_name,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (14) Sort [cs_bill_customer_sk,cs_item_sk] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) + Exchange [cs_bill_customer_sk] #8 + WholeStageCodegen (13) Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_bill_customer_sk,cs_item_sk] @@ -100,4 +93,4 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index fc55789fab16a..b5bee8dc1213c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -1,53 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * SortMergeJoin Inner (44) - :- * Sort (35) - : +- Exchange (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.store (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (31) - : +- Exchange (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.store_returns (24) - : +- ReusedExchange (27) - +- * Sort (43) - +- Exchange (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.catalog_sales (36) - +- ReusedExchange (39) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Project (32) + : +- * SortMergeJoin Inner (31) + : :- * Sort (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.store (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (30) + : +- Exchange (29) + : +- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.store_returns (23) + : +- ReusedExchange (26) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet default.catalog_sales (33) + +- ReusedExchange (36) (1) Scan parquet default.store_sales @@ -65,7 +62,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#8] (5) BroadcastHashJoin [codegen id : 3] @@ -143,194 +140,200 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_sk#14, i_item_id#15, i_item_desc#16] -(22) Exchange -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [id=#18] - -(23) Sort [codegen id : 8] +(22) Sort [codegen id : 7] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] Arguments: [ss_customer_sk#2 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.store_returns -Output [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23] +(23) Scan parquet default.store_returns +Output [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#23), dynamicpruningexpression(sr_returned_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(sr_returned_date_sk#22), dynamicpruningexpression(sr_returned_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 10] -Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23] +(24) ColumnarToRow [codegen id : 9] +Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22] -(26) Filter [codegen id : 10] -Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23] -Condition : ((isnotnull(sr_customer_sk#20) AND isnotnull(sr_item_sk#19)) AND isnotnull(sr_ticket_number#21)) +(25) Filter [codegen id : 9] +Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22] +Condition : ((isnotnull(sr_customer_sk#19) AND isnotnull(sr_item_sk#18)) AND isnotnull(sr_ticket_number#20)) -(27) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#25] +(26) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#24] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [d_date_sk#25] +(27) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [sr_returned_date_sk#22] +Right keys [1]: [d_date_sk#24] Join condition: None -(29) Project [codegen id : 10] -Output [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22] -Input [6]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] +(28) Project [codegen id : 9] +Output [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21] +Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21, sr_returned_date_sk#22, d_date_sk#24] -(30) Exchange -Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22] -Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21, 5), ENSURE_REQUIREMENTS, [id=#26] +(29) Exchange +Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21] +Arguments: hashpartitioning(sr_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#25] -(31) Sort [codegen id : 11] -Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22] -Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST, sr_ticket_number#21 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 10] +Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21] +Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST, sr_ticket_number#20 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 12] +(31) SortMergeJoin [codegen id : 11] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21] +Right keys [3]: [sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20] Join condition: None -(33) Project [codegen id : 12] -Output [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_net_loss#22] - -(34) Exchange -Input [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22] -Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#27] - -(35) Sort [codegen id : 13] -Input [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22] -Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST], false, 0 +(32) Project [codegen id : 11] +Output [8]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_net_loss#21] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21] -(36) Scan parquet default.catalog_sales -Output [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31] +(33) Scan parquet default.catalog_sales +Output [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(cs_sold_date_sk#29), dynamicpruningexpression(cs_sold_date_sk#29 IN dynamicpruning#23)] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 15] -Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31] +(34) ColumnarToRow [codegen id : 13] +Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29] -(38) Filter [codegen id : 15] -Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31] -Condition : (isnotnull(cs_bill_customer_sk#28) AND isnotnull(cs_item_sk#29)) +(35) Filter [codegen id : 13] +Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29] +Condition : (isnotnull(cs_bill_customer_sk#26) AND isnotnull(cs_item_sk#27)) -(39) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#32] +(36) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#30] -(40) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#31] -Right keys [1]: [d_date_sk#32] +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None -(41) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30] -Input [5]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#32] +(38) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28] +Input [5]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_date_sk#29, d_date_sk#30] -(42) Exchange -Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30] -Arguments: hashpartitioning(cs_bill_customer_sk#28, cs_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#33] +(39) Exchange +Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28] +Arguments: hashpartitioning(cs_bill_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] -(43) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30] -Arguments: [cs_bill_customer_sk#28 ASC NULLS FIRST, cs_item_sk#29 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28] +Arguments: [cs_bill_customer_sk#26 ASC NULLS FIRST, cs_item_sk#27 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#20, sr_item_sk#19] -Right keys [2]: [cs_bill_customer_sk#28, cs_item_sk#29] +(41) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#19, sr_item_sk#18] +Right keys [2]: [cs_bill_customer_sk#26, cs_item_sk#27] Join condition: None -(45) Project [codegen id : 17] -Output [7]: [ss_net_profit#5, sr_net_loss#22, cs_net_profit#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] -Input [11]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_net_loss#22, cs_bill_customer_sk#28, cs_item_sk#29, cs_net_profit#30] +(42) Project [codegen id : 15] +Output [7]: [ss_net_profit#5, sr_net_loss#21, cs_net_profit#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] +Input [11]: [ss_net_profit#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_net_loss#21, cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28] -(46) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#5, sr_net_loss#22, cs_net_profit#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] +(43) HashAggregate [codegen id : 15] +Input [7]: [ss_net_profit#5, sr_net_loss#21, cs_net_profit#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#22)), partial_sum(UnscaledValue(cs_net_profit#30))] -Aggregate Attributes [3]: [sum#34, sum#35, sum#36] -Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#37, sum#38, sum#39] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#21)), partial_sum(UnscaledValue(cs_net_profit#28))] +Aggregate Attributes [3]: [sum#32, sum#33, sum#34] +Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#35, sum#36, sum#37] -(47) Exchange -Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#37, sum#38, sum#39] -Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#40] +(44) Exchange +Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#35, sum#36, sum#37] +Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#38] -(48) HashAggregate [codegen id : 18] -Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#37, sum#38, sum#39] +(45) HashAggregate [codegen id : 16] +Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#35, sum#36, sum#37] Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#22)), sum(UnscaledValue(cs_net_profit#30))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#41, sum(UnscaledValue(sr_net_loss#22))#42, sum(UnscaledValue(cs_net_profit#30))#43] -Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#41,17,2) AS store_sales_profit#44, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#42,17,2) AS store_returns_loss#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#30))#43,17,2) AS catalog_sales_profit#46] +Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#21)), sum(UnscaledValue(cs_net_profit#28))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#39, sum(UnscaledValue(sr_net_loss#21))#40, sum(UnscaledValue(cs_net_profit#28))#41] +Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#39,17,2) AS store_sales_profit#42, MakeDecimal(sum(UnscaledValue(sr_net_loss#21))#40,17,2) AS store_returns_loss#43, MakeDecimal(sum(UnscaledValue(cs_net_profit#28))#41,17,2) AS catalog_sales_profit#44] -(49) TakeOrderedAndProject -Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46] -Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46] +(46) TakeOrderedAndProject +Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#42, store_returns_loss#43, catalog_sales_profit#44] +Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#42, store_returns_loss#43, catalog_sales_profit#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (54) -+- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) +BroadcastExchange (51) ++- * Project (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.date_dim (47) -(50) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#47, d_moy#48] +(47) Scan parquet default.date_dim +Output [3]: [d_date_sk#8, d_year#45, d_moy#46] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#47, d_moy#48] +(48) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#8, d_year#45, d_moy#46] -(52) Filter [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#47, d_moy#48] -Condition : ((((isnotnull(d_moy#48) AND isnotnull(d_year#47)) AND (d_moy#48 = 4)) AND (d_year#47 = 2001)) AND isnotnull(d_date_sk#8)) +(49) Filter [codegen id : 1] +Input [3]: [d_date_sk#8, d_year#45, d_moy#46] +Condition : ((((isnotnull(d_moy#46) AND isnotnull(d_year#45)) AND (d_moy#46 = 4)) AND (d_year#45 = 2001)) AND isnotnull(d_date_sk#8)) -(53) Project [codegen id : 1] +(50) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [3]: [d_date_sk#8, d_year#47, d_moy#48] +Input [3]: [d_date_sk#8, d_year#45, d_moy#46] -(54) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] - -Subquery:2 Hosting operator id = 24 Hosting Expression = sr_returned_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet default.date_dim (55) - - -(55) Scan parquet default.date_dim -Output [3]: [d_date_sk#25, d_year#50, d_moy#51] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#49] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (56) ++- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet default.date_dim (52) + + +(52) Scan parquet default.date_dim +Output [3]: [d_date_sk#24, d_year#48, d_moy#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#25, d_year#50, d_moy#51] +(53) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#24, d_year#48, d_moy#49] -(57) Filter [codegen id : 1] -Input [3]: [d_date_sk#25, d_year#50, d_moy#51] -Condition : (((((isnotnull(d_moy#51) AND isnotnull(d_year#50)) AND (d_moy#51 >= 4)) AND (d_moy#51 <= 10)) AND (d_year#50 = 2001)) AND isnotnull(d_date_sk#25)) +(54) Filter [codegen id : 1] +Input [3]: [d_date_sk#24, d_year#48, d_moy#49] +Condition : (((((isnotnull(d_moy#49) AND isnotnull(d_year#48)) AND (d_moy#49 >= 4)) AND (d_moy#49 <= 10)) AND (d_year#48 = 2001)) AND isnotnull(d_date_sk#24)) -(58) Project [codegen id : 1] -Output [1]: [d_date_sk#25] -Input [3]: [d_date_sk#25, d_year#50, d_moy#51] +(55) Project [codegen id : 1] +Output [1]: [d_date_sk#24] +Input [3]: [d_date_sk#24, d_year#48, d_moy#49] +<<<<<<< HEAD (59) BroadcastExchange Input [1]: [d_date_sk#25] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] - -Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#24 +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#52] +======= +(56) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index 23d7e84027b2e..d0604e056b7cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -1,97 +1,90 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (18) + WholeStageCodegen (16) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] InputAdapter Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (17) + WholeStageCodegen (15) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] + WholeStageCodegen (11) + Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (7) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] - InputAdapter - WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 - WholeStageCodegen (10) - Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (10) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk] #6 + WholeStageCodegen (9) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (14) Sort [cs_bill_customer_sk,cs_item_sk] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) + Exchange [cs_bill_customer_sk] #8 + WholeStageCodegen (13) Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_bill_customer_sk,cs_item_sk] @@ -100,4 +93,4 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index 221439075d24d..7c12f3ca8d1e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -1,53 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * SortMergeJoin Inner (44) - :- * Sort (35) - : +- Exchange (34) - : +- * Project (33) - : +- * SortMergeJoin Inner (32) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Project (21) - : : +- * SortMergeJoin Inner (20) - : : :- * Sort (14) - : : : +- Exchange (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * Filter (9) - : : : +- * ColumnarToRow (8) - : : : +- Scan parquet default.store (7) - : : +- * Sort (19) - : : +- Exchange (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.item (15) - : +- * Sort (31) - : +- Exchange (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.store_returns (24) - : +- ReusedExchange (27) - +- * Sort (43) - +- Exchange (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Filter (38) - : +- * ColumnarToRow (37) - : +- Scan parquet default.catalog_sales (36) - +- ReusedExchange (39) +TakeOrderedAndProject (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Project (32) + : +- * SortMergeJoin Inner (31) + : :- * Sort (22) + : : +- * Project (21) + : : +- * SortMergeJoin Inner (20) + : : :- * Sort (14) + : : : +- Exchange (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * Filter (9) + : : : +- * ColumnarToRow (8) + : : : +- Scan parquet default.store (7) + : : +- * Sort (19) + : : +- Exchange (18) + : : +- * Filter (17) + : : +- * ColumnarToRow (16) + : : +- Scan parquet default.item (15) + : +- * Sort (30) + : +- Exchange (29) + : +- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Filter (25) + : : +- * ColumnarToRow (24) + : : +- Scan parquet default.store_returns (23) + : +- ReusedExchange (26) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Filter (35) + : +- * ColumnarToRow (34) + : +- Scan parquet default.catalog_sales (33) + +- ReusedExchange (36) (1) Scan parquet default.store_sales @@ -65,7 +62,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) -(4) ReusedExchange [Reuses operator id: 54] +(4) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#8] (5) BroadcastHashJoin [codegen id : 3] @@ -143,192 +140,190 @@ Join condition: None Output [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_sk#14, i_item_id#15, i_item_desc#16] -(22) Exchange -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [id=#18] - -(23) Sort [codegen id : 8] +(22) Sort [codegen id : 7] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] Arguments: [ss_customer_sk#2 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0 -(24) Scan parquet default.store_returns -Output [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23] +(23) Scan parquet default.store_returns +Output [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#23), dynamicpruningexpression(sr_returned_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(sr_returned_date_sk#22), dynamicpruningexpression(sr_returned_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 10] -Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23] +(24) ColumnarToRow [codegen id : 9] +Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22] -(26) Filter [codegen id : 10] -Input [5]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23] -Condition : ((isnotnull(sr_customer_sk#20) AND isnotnull(sr_item_sk#19)) AND isnotnull(sr_ticket_number#21)) +(25) Filter [codegen id : 9] +Input [5]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22] +Condition : ((isnotnull(sr_customer_sk#19) AND isnotnull(sr_item_sk#18)) AND isnotnull(sr_ticket_number#20)) -(27) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#25] +(26) ReusedExchange [Reuses operator id: 56] +Output [1]: [d_date_sk#24] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [d_date_sk#25] +(27) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [sr_returned_date_sk#22] +Right keys [1]: [d_date_sk#24] Join condition: None -(29) Project [codegen id : 10] -Output [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22] -Input [6]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22, sr_returned_date_sk#23, d_date_sk#25] +(28) Project [codegen id : 9] +Output [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] +Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21, sr_returned_date_sk#22, d_date_sk#24] -(30) Exchange -Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22] -Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21, 5), ENSURE_REQUIREMENTS, [id=#26] +(29) Exchange +Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] +Arguments: hashpartitioning(sr_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#25] -(31) Sort [codegen id : 11] -Input [4]: [sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22] -Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST, sr_ticket_number#21 ASC NULLS FIRST], false, 0 +(30) Sort [codegen id : 10] +Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] +Arguments: [sr_customer_sk#19 ASC NULLS FIRST, sr_item_sk#18 ASC NULLS FIRST, sr_ticket_number#20 ASC NULLS FIRST], false, 0 -(32) SortMergeJoin [codegen id : 12] +(31) SortMergeJoin [codegen id : 11] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#20, sr_item_sk#19, sr_ticket_number#21] +Right keys [3]: [sr_customer_sk#19, sr_item_sk#18, sr_ticket_number#20] Join condition: None -(33) Project [codegen id : 12] -Output [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_ticket_number#21, sr_return_quantity#22] - -(34) Exchange -Input [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22] -Arguments: hashpartitioning(sr_customer_sk#20, sr_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#27] - -(35) Sort [codegen id : 13] -Input [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22] -Arguments: [sr_customer_sk#20 ASC NULLS FIRST, sr_item_sk#19 ASC NULLS FIRST], false, 0 +(32) Project [codegen id : 11] +Output [8]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] -(36) Scan parquet default.catalog_sales -Output [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31] +(33) Scan parquet default.catalog_sales +Output [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(cs_sold_date_sk#29), dynamicpruningexpression(cs_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 15] -Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31] +(34) ColumnarToRow [codegen id : 13] +Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29] -(38) Filter [codegen id : 15] -Input [4]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31] -Condition : (isnotnull(cs_bill_customer_sk#28) AND isnotnull(cs_item_sk#29)) +(35) Filter [codegen id : 13] +Input [4]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29] +Condition : (isnotnull(cs_bill_customer_sk#26) AND isnotnull(cs_item_sk#27)) -(39) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#33] +(36) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#31] -(40) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#31] -Right keys [1]: [d_date_sk#33] +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None -(41) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30] -Input [5]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30, cs_sold_date_sk#31, d_date_sk#33] +(38) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28] +Input [5]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_sk#29, d_date_sk#31] -(42) Exchange -Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30] -Arguments: hashpartitioning(cs_bill_customer_sk#28, cs_item_sk#29, 5), ENSURE_REQUIREMENTS, [id=#34] +(39) Exchange +Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28] +Arguments: hashpartitioning(cs_bill_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] -(43) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30] -Arguments: [cs_bill_customer_sk#28 ASC NULLS FIRST, cs_item_sk#29 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28] +Arguments: [cs_bill_customer_sk#26 ASC NULLS FIRST, cs_item_sk#27 ASC NULLS FIRST], false, 0 -(44) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#20, sr_item_sk#19] -Right keys [2]: [cs_bill_customer_sk#28, cs_item_sk#29] +(41) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#19, sr_item_sk#18] +Right keys [2]: [cs_bill_customer_sk#26, cs_item_sk#27] Join condition: None -(45) Project [codegen id : 17] -Output [7]: [ss_quantity#5, sr_return_quantity#22, cs_quantity#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] -Input [11]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#19, sr_customer_sk#20, sr_return_quantity#22, cs_bill_customer_sk#28, cs_item_sk#29, cs_quantity#30] +(42) Project [codegen id : 15] +Output [7]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] +Input [11]: [ss_quantity#5, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16, sr_item_sk#18, sr_customer_sk#19, sr_return_quantity#21, cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28] -(46) HashAggregate [codegen id : 17] -Input [7]: [ss_quantity#5, sr_return_quantity#22, cs_quantity#30, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] +(43) HashAggregate [codegen id : 15] +Input [7]: [ss_quantity#5, sr_return_quantity#21, cs_quantity#28, s_store_id#10, s_store_name#11, i_item_id#15, i_item_desc#16] Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#22), partial_sum(cs_quantity#30)] -Aggregate Attributes [3]: [sum#35, sum#36, sum#37] -Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#38, sum#39, sum#40] +Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#21), partial_sum(cs_quantity#28)] +Aggregate Attributes [3]: [sum#33, sum#34, sum#35] +Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#36, sum#37, sum#38] -(47) Exchange -Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#38, sum#39, sum#40] -Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#41] +(44) Exchange +Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#36, sum#37, sum#38] +Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, 5), ENSURE_REQUIREMENTS, [id=#39] -(48) HashAggregate [codegen id : 18] -Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#38, sum#39, sum#40] +(45) HashAggregate [codegen id : 16] +Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum#36, sum#37, sum#38] Keys [4]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#22), sum(cs_quantity#30)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#42, sum(sr_return_quantity#22)#43, sum(cs_quantity#30)#44] -Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum(ss_quantity#5)#42 AS store_sales_quantity#45, sum(sr_return_quantity#22)#43 AS store_returns_quantity#46, sum(cs_quantity#30)#44 AS catalog_sales_quantity#47] +Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#21), sum(cs_quantity#28)] +Aggregate Attributes [3]: [sum(ss_quantity#5)#40, sum(sr_return_quantity#21)#41, sum(cs_quantity#28)#42] +Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum(ss_quantity#5)#40 AS store_sales_quantity#43, sum(sr_return_quantity#21)#41 AS store_returns_quantity#44, sum(cs_quantity#28)#42 AS catalog_sales_quantity#45] -(49) TakeOrderedAndProject -Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47] -Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47] +(46) TakeOrderedAndProject +Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#43, store_returns_quantity#44, catalog_sales_quantity#45] +Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#43, store_returns_quantity#44, catalog_sales_quantity#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (54) -+- * Project (53) - +- * Filter (52) - +- * ColumnarToRow (51) - +- Scan parquet default.date_dim (50) +BroadcastExchange (51) ++- * Project (50) + +- * Filter (49) + +- * ColumnarToRow (48) + +- Scan parquet default.date_dim (47) -(50) Scan parquet default.date_dim -Output [3]: [d_date_sk#8, d_year#48, d_moy#49] +(47) Scan parquet default.date_dim +Output [3]: [d_date_sk#8, d_year#46, d_moy#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#48, d_moy#49] +(48) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#8, d_year#46, d_moy#47] -(52) Filter [codegen id : 1] -Input [3]: [d_date_sk#8, d_year#48, d_moy#49] -Condition : ((((isnotnull(d_moy#49) AND isnotnull(d_year#48)) AND (d_moy#49 = 9)) AND (d_year#48 = 1999)) AND isnotnull(d_date_sk#8)) +(49) Filter [codegen id : 1] +Input [3]: [d_date_sk#8, d_year#46, d_moy#47] +Condition : ((((isnotnull(d_moy#47) AND isnotnull(d_year#46)) AND (d_moy#47 = 9)) AND (d_year#46 = 1999)) AND isnotnull(d_date_sk#8)) -(53) Project [codegen id : 1] +(50) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [3]: [d_date_sk#8, d_year#48, d_moy#49] +Input [3]: [d_date_sk#8, d_year#46, d_moy#47] -(54) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] - -Subquery:2 Hosting operator id = 24 Hosting Expression = sr_returned_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (59) -+- * Project (58) - +- * Filter (57) - +- * ColumnarToRow (56) - +- Scan parquet default.date_dim (55) - - -(55) Scan parquet default.date_dim -Output [3]: [d_date_sk#25, d_year#51, d_moy#52] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#50] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (56) ++- * Project (55) + +- * Filter (54) + +- * ColumnarToRow (53) + +- Scan parquet default.date_dim (52) + + +(52) Scan parquet default.date_dim +Output [3]: [d_date_sk#24, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#25, d_year#51, d_moy#52] +(53) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#24, d_year#49, d_moy#50] -(57) Filter [codegen id : 1] -Input [3]: [d_date_sk#25, d_year#51, d_moy#52] -Condition : (((((isnotnull(d_moy#52) AND isnotnull(d_year#51)) AND (d_moy#52 >= 9)) AND (d_moy#52 <= 12)) AND (d_year#51 = 1999)) AND isnotnull(d_date_sk#25)) +(54) Filter [codegen id : 1] +Input [3]: [d_date_sk#24, d_year#49, d_moy#50] +Condition : (((((isnotnull(d_moy#50) AND isnotnull(d_year#49)) AND (d_moy#50 >= 9)) AND (d_moy#50 <= 12)) AND (d_year#49 = 1999)) AND isnotnull(d_date_sk#24)) -(58) Project [codegen id : 1] -Output [1]: [d_date_sk#25] -Input [3]: [d_date_sk#25, d_year#51, d_moy#52] +(55) Project [codegen id : 1] +Output [1]: [d_date_sk#24] +Input [3]: [d_date_sk#24, d_year#49, d_moy#50] +<<<<<<< HEAD (59) BroadcastExchange Input [1]: [d_date_sk#25] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#32 @@ -341,24 +336,53 @@ BroadcastExchange (64) (60) Scan parquet default.date_dim Output [2]: [d_date_sk#33, d_year#54] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] +======= +(56) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +>>>>>>> 7190faa7a9 (update manifest files) + +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (61) ++- * Project (60) + +- * Filter (59) + +- * ColumnarToRow (58) + +- Scan parquet default.date_dim (57) + + +(57) Scan parquet default.date_dim +Output [2]: [d_date_sk#31, d_year#52] +>>>>>>> 10c61d45bc (update manifest files) Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(61) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#54] +(58) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#52] -(62) Filter [codegen id : 1] -Input [2]: [d_date_sk#33, d_year#54] -Condition : (d_year#54 IN (1999,2000,2001) AND isnotnull(d_date_sk#33)) +(59) Filter [codegen id : 1] +Input [2]: [d_date_sk#31, d_year#52] +Condition : (d_year#52 IN (1999,2000,2001) AND isnotnull(d_date_sk#31)) -(63) Project [codegen id : 1] -Output [1]: [d_date_sk#33] -Input [2]: [d_date_sk#33, d_year#54] +(60) Project [codegen id : 1] +Output [1]: [d_date_sk#31] +Input [2]: [d_date_sk#31, d_year#52] +<<<<<<< HEAD (64) BroadcastExchange Input [1]: [d_date_sk#33] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#55] +======= +(61) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt index 5463f3f0a8fd4..4a2f65b44adc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt @@ -1,97 +1,90 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (18) + WholeStageCodegen (16) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] InputAdapter Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (17) + WholeStageCodegen (15) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] - InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] + WholeStageCodegen (11) + Project [ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (7) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] - InputAdapter - WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] - InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 - WholeStageCodegen (10) - Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (10) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk] #6 + WholeStageCodegen (9) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter - WholeStageCodegen (16) + WholeStageCodegen (14) Sort [cs_bill_customer_sk,cs_item_sk] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) + Exchange [cs_bill_customer_sk] #8 + WholeStageCodegen (13) Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_bill_customer_sk,cs_item_sk] @@ -99,7 +92,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales InputAdapter Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #11 + BroadcastExchange #9 WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] @@ -107,4 +100,4 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index 44a956471b61e..c7a150e7621be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -289,6 +289,22 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +======= +<<<<<<< HEAD +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#51] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index ad356d44af668..1feb8125a359b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -289,6 +289,22 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +======= +<<<<<<< HEAD +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index c6971f3ea904b..836dfabaeaa36 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -1,74 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * SortMergeJoin Inner (49) - : : :- * Sort (37) - : : : +- Exchange (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- ReusedExchange (33) - : : +- * Sort (48) - : : +- Exchange (47) - : : +- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.inventory (38) - : : +- BroadcastExchange (44) - : : +- * Filter (43) - : : +- * ColumnarToRow (42) - : : +- Scan parquet default.warehouse (41) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * ColumnarToRow (52) - : +- Scan parquet default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet default.catalog_returns (59) +TakeOrderedAndProject (68) ++- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * SortMergeJoin LeftOuter (63) + :- * Sort (56) + : +- * Project (55) + : +- * BroadcastHashJoin LeftOuter BuildRight (54) + : :- * Project (49) + : : +- * SortMergeJoin Inner (48) + : : :- * Sort (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * Project (32) + : : : : +- * SortMergeJoin Inner (31) + : : : : :- * Sort (25) + : : : : : +- Exchange (24) + : : : : : +- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Project (17) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : : :- * Project (10) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : : :- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (8) + : : : : : : : +- * Project (7) + : : : : : : : +- * Filter (6) + : : : : : : : +- * ColumnarToRow (5) + : : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : : +- BroadcastExchange (15) + : : : : : : +- * Project (14) + : : : : : : +- * Filter (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- Scan parquet default.customer_demographics (11) + : : : : : +- BroadcastExchange (21) + : : : : : +- * Filter (20) + : : : : : +- * ColumnarToRow (19) + : : : : : +- Scan parquet default.date_dim (18) + : : : : +- * Sort (30) + : : : : +- Exchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- ReusedExchange (33) + : : +- * Sort (47) + : : +- Exchange (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- * Filter (39) + : : : +- * ColumnarToRow (38) + : : : +- Scan parquet default.inventory (37) + : : +- BroadcastExchange (43) + : : +- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.warehouse (40) + : +- BroadcastExchange (53) + : +- * Filter (52) + : +- * ColumnarToRow (51) + : +- Scan parquet default.promotion (50) + +- * Sort (62) + +- Exchange (61) + +- * Project (60) + +- * Filter (59) + +- * ColumnarToRow (58) + +- Scan parquet default.catalog_returns (57) (1) Scan parquet default.catalog_sales @@ -214,7 +212,7 @@ Join condition: None Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21] Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21] -(33) ReusedExchange [Reuses operator id: 81] +(33) ReusedExchange [Reuses operator id: 79] Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] (34) BroadcastHashJoin [codegen id : 10] @@ -226,228 +224,260 @@ Join condition: (d_date#17 > date_add(d_date#24, 5)) Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -(36) Exchange -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#26, 5), ENSURE_REQUIREMENTS, [id=#27] - -(37) Sort [codegen id : 11] +(36) Sort [codegen id : 10] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#26 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.inventory -Output [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(37) Scan parquet default.inventory +Output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#31), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(38) ColumnarToRow [codegen id : 12] +Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] -(40) Filter [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] -Condition : ((isnotnull(inv_quantity_on_hand#30) AND isnotnull(inv_item_sk#28)) AND isnotnull(inv_warehouse_sk#29)) +(39) Filter [codegen id : 12] +Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] +Condition : ((isnotnull(inv_quantity_on_hand#29) AND isnotnull(inv_item_sk#27)) AND isnotnull(inv_warehouse_sk#28)) -(41) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(40) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(41) ColumnarToRow [codegen id : 11] +Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -(43) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -Condition : isnotnull(w_warehouse_sk#32) +(42) Filter [codegen id : 11] +Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +Condition : isnotnull(w_warehouse_sk#31) +<<<<<<< HEAD (44) BroadcastExchange Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (45) BroadcastHashJoin [codegen id : 13] Left keys [1]: [inv_warehouse_sk#29] Right keys [1]: [w_warehouse_sk#32] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#34] +======= +(43) BroadcastExchange +Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +>>>>>>> 7190faa7a9 (update manifest files) + +(44) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [inv_warehouse_sk#28] +Right keys [1]: [w_warehouse_sk#31] +>>>>>>> 10c61d45bc (update manifest files) Join condition: None -(46) Project [codegen id : 13] -Output [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Input [6]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(45) Project [codegen id : 12] +Output [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] +Input [6]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_sk#31, w_warehouse_name#32] -(47) Exchange -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: hashpartitioning(inv_item_sk#28, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#35] +(46) Exchange +Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] +Arguments: hashpartitioning(inv_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#34] -(48) Sort [codegen id : 14] -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: [inv_item_sk#28 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +(47) Sort [codegen id : 13] +Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] +Arguments: [inv_item_sk#27 ASC NULLS FIRST, inv_date_sk#30 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] +(48) SortMergeJoin [codegen id : 15] Left keys [2]: [cs_item_sk#4, d_date_sk#26] -Right keys [2]: [inv_item_sk#28, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#30 < cs_quantity#7) +Right keys [2]: [inv_item_sk#27, inv_date_sk#30] +Join condition: (inv_quantity_on_hand#29 < cs_quantity#7) -(50) Project [codegen id : 16] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] +(49) Project [codegen id : 15] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] -(51) Scan parquet default.promotion -Output [1]: [p_promo_sk#36] +(50) Scan parquet default.promotion +Output [1]: [p_promo_sk#35] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 15] -Input [1]: [p_promo_sk#36] +(51) ColumnarToRow [codegen id : 14] +Input [1]: [p_promo_sk#35] -(53) Filter [codegen id : 15] -Input [1]: [p_promo_sk#36] -Condition : isnotnull(p_promo_sk#36) +(52) Filter [codegen id : 14] +Input [1]: [p_promo_sk#35] +Condition : isnotnull(p_promo_sk#35) +<<<<<<< HEAD (54) BroadcastExchange Input [1]: [p_promo_sk#36] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] - -(55) BroadcastHashJoin [codegen id : 16] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#37] +======= +(53) BroadcastExchange +Input [1]: [p_promo_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +(54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#36] +Right keys [1]: [p_promo_sk#35] Join condition: None -(56) Project [codegen id : 16] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, p_promo_sk#36] +(55) Project [codegen id : 15] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, p_promo_sk#35] -(57) Exchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38] - -(58) Sort [codegen id : 17] -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] +(56) Sort [codegen id : 15] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(57) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(58) ColumnarToRow [codegen id : 16] +Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] -(61) Filter [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) +(59) Filter [codegen id : 16] +Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] +Condition : (isnotnull(cr_item_sk#37) AND isnotnull(cr_order_number#38)) -(62) Project [codegen id : 18] -Output [2]: [cr_item_sk#39, cr_order_number#40] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(60) Project [codegen id : 16] +Output [2]: [cr_item_sk#37, cr_order_number#38] +Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] -(63) Exchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42] +(61) Exchange +Input [2]: [cr_item_sk#37, cr_order_number#38] +Arguments: hashpartitioning(cr_item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#40] -(64) Sort [codegen id : 19] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 17] +Input [2]: [cr_item_sk#37, cr_order_number#38] +Arguments: [cr_item_sk#37 ASC NULLS FIRST, cr_order_number#38 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 20] +(63) SortMergeJoin [codegen id : 18] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] +Right keys [2]: [cr_item_sk#37, cr_order_number#38] Join condition: None -(66) Project [codegen id : 20] -Output [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, cr_item_sk#39, cr_order_number#40] +(64) Project [codegen id : 18] +Output [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, cr_item_sk#37, cr_order_number#38] -(67) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(65) HashAggregate [codegen id : 18] +Input [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] +Aggregate Attributes [1]: [count#41] +Results [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42] -(68) Exchange -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#33, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#45] +(66) Exchange +Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42] +Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#32, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#43] -(69) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(67) HashAggregate [codegen id : 19] +Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42] +Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49] +Aggregate Attributes [1]: [count(1)#44] +Results [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count(1)#44 AS no_promo#45, count(1)#44 AS promo#46, count(1)#44 AS total_cnt#47] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] +(68) TakeOrderedAndProject +Input [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47] +Arguments: 100, [total_cnt#47 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#32 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (81) -+- * Project (80) - +- * BroadcastHashJoin Inner BuildLeft (79) - :- BroadcastExchange (75) - : +- * Project (74) - : +- * Filter (73) - : +- * ColumnarToRow (72) - : +- Scan parquet default.date_dim (71) - +- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet default.date_dim (76) - - -(71) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] +BroadcastExchange (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildLeft (77) + :- BroadcastExchange (73) + : +- * Project (72) + : +- * Filter (71) + : +- * ColumnarToRow (70) + : +- Scan parquet default.date_dim (69) + +- * Filter (76) + +- * ColumnarToRow (75) + +- Scan parquet default.date_dim (74) + + +(69) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] +(70) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] -(73) Filter [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] -Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) +(71) Filter [codegen id : 1] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] +Condition : ((((isnotnull(d_year#48) AND (d_year#48 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(74) Project [codegen id : 1] +(72) Project [codegen id : 1] Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] -(75) BroadcastExchange +(73) BroadcastExchange Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] - -(76) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#52] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#51] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +(74) Scan parquet default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(77) ColumnarToRow -Input [2]: [d_date_sk#26, d_week_seq#52] +(75) ColumnarToRow +Input [2]: [d_date_sk#26, d_week_seq#50] -(78) Filter -Input [2]: [d_date_sk#26, d_week_seq#52] -Condition : (isnotnull(d_week_seq#52) AND isnotnull(d_date_sk#26)) +(76) Filter +Input [2]: [d_date_sk#26, d_week_seq#50] +Condition : (isnotnull(d_week_seq#50) AND isnotnull(d_date_sk#26)) -(79) BroadcastHashJoin [codegen id : 2] +(77) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#52] +Right keys [1]: [d_week_seq#50] Join condition: None -(80) Project [codegen id : 2] +(78) Project [codegen id : 2] Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#52] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50] -(81) BroadcastExchange +(79) BroadcastExchange Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index e838025a71db8..d84393b2ff106 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -1,132 +1,126 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (21) + WholeStageCodegen (19) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (20) + WholeStageCodegen (18) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (15) Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (16) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] - InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] - InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + InputAdapter + WholeStageCodegen (10) + Sort [cs_item_sk,d_date_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #2 + WholeStageCodegen (4) + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #10 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] + InputAdapter + ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #3 + InputAdapter + WholeStageCodegen (13) + Sort [inv_item_sk,inv_date_sk] InputAdapter - WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] - InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 - WholeStageCodegen (13) - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + Exchange [inv_item_sk] #9 + WholeStageCodegen (12) + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (11) + Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (12) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (15) - Filter [p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (14) + Filter [p_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (19) + WholeStageCodegen (17) Sort [cr_item_sk,cr_order_number] InputAdapter - Exchange [cr_item_sk,cr_order_number] #14 - WholeStageCodegen (18) + Exchange [cr_item_sk] #12 + WholeStageCodegen (16) Project [cr_item_sk,cr_order_number] Filter [cr_item_sk,cr_order_number] ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index c08379b07b397..e10cef938901f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -273,34 +273,37 @@ Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_s ===== Subqueries ===== Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (76) -+- Exchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * SortMergeJoin Inner (69) - :- * Sort (66) - : +- Exchange (65) - : +- * Project (64) - : +- * SortMergeJoin Inner (63) - : :- * Sort (57) - : : +- Exchange (56) - : : +- * Project (55) - : : +- * BroadcastHashJoin Inner BuildLeft (54) - : : :- ReusedExchange (49) - : : +- * Project (53) - : : +- * Filter (52) - : : +- * ColumnarToRow (51) - : : +- Scan parquet default.store_sales (50) - : +- * Sort (62) - : +- Exchange (61) - : +- * Filter (60) - : +- * ColumnarToRow (59) - : +- Scan parquet default.item (58) - +- * Sort (68) - +- ReusedExchange (67) +* HashAggregate (79) ++- Exchange (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * SortMergeJoin Inner (72) + :- * Sort (65) + : +- * Project (64) + : +- * SortMergeJoin Inner (63) + : :- * Sort (57) + : : +- Exchange (56) + : : +- * Project (55) + : : +- * BroadcastHashJoin Inner BuildLeft (54) + : : :- ReusedExchange (49) + : : +- * Project (53) + : : +- * Filter (52) + : : +- * ColumnarToRow (51) + : : +- Scan parquet default.store_sales (50) + : +- * Sort (62) + : +- Exchange (61) + : +- * Filter (60) + : +- * ColumnarToRow (59) + : +- Scan parquet default.item (58) + +- * Sort (71) + +- Exchange (70) + +- * Project (69) + +- * Filter (68) + +- * ColumnarToRow (67) + +- Scan parquet default.store_returns (66) (49) ReusedExchange [Reuses operator id: 17] @@ -372,60 +375,75 @@ Join condition: None Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -(65) Exchange -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: hashpartitioning(ss_ticket_number#21, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#53] - -(66) Sort [codegen id : 9] +(65) Sort [codegen id : 8] Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] Arguments: [ss_ticket_number#21 ASC NULLS FIRST, ss_item_sk#18 ASC NULLS FIRST], false, 0 -(67) ReusedExchange [Reuses operator id: 36] +(66) Scan parquet default.store_returns +Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct + +(67) ColumnarToRow [codegen id : 9] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] + +(68) Filter [codegen id : 9] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) + +(69) Project [codegen id : 9] Output [2]: [sr_item_sk#32, sr_ticket_number#33] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] -(68) Sort [codegen id : 11] +(70) Exchange +Input [2]: [sr_item_sk#32, sr_ticket_number#33] +Arguments: hashpartitioning(sr_ticket_number#33, 5), ENSURE_REQUIREMENTS, [id=#53] + +(71) Sort [codegen id : 10] Input [2]: [sr_item_sk#32, sr_ticket_number#33] Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 12] +(72) SortMergeJoin [codegen id : 11] Left keys [2]: [ss_ticket_number#21, ss_item_sk#18] Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None -(70) Project [codegen id : 12] +(73) Project [codegen id : 11] Output [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, sr_item_sk#32, sr_ticket_number#33] -(71) HashAggregate [codegen id : 12] +(74) HashAggregate [codegen id : 11] Input [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] Aggregate Attributes [1]: [sum#54] Results [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55] -(72) Exchange +(75) Exchange Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55] Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, 5), ENSURE_REQUIREMENTS, [id=#56] -(73) HashAggregate [codegen id : 13] +(76) HashAggregate [codegen id : 12] Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55] Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 13] +(77) HashAggregate [codegen id : 12] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] Aggregate Attributes [2]: [sum#57, count#58] Results [2]: [sum#59, count#60] -(75) Exchange +(78) Exchange Input [2]: [sum#59, count#60] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] -(76) HashAggregate [codegen id : 14] +(79) HashAggregate [codegen id : 13] Input [2]: [sum#59, count#60] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index 4beebcbbe52ef..a83743673a9c4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -5,57 +5,60 @@ WholeStageCodegen (12) WholeStageCodegen (11) Filter [paid] Subquery #1 - WholeStageCodegen (14) + WholeStageCodegen (13) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] InputAdapter Exchange #10 - WholeStageCodegen (13) + WholeStageCodegen (12) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (8) Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #12 - WholeStageCodegen (8) - Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #13 - WholeStageCodegen (4) - Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] - BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5 - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_item_sk] InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #14 - WholeStageCodegen (6) - Filter [i_item_sk] + Exchange [ss_item_sk] #12 + WholeStageCodegen (4) + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] + BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] #5 + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #13 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (10) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number] #9 + Exchange [sr_ticket_number] #14 + WholeStageCodegen (9) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 51b2f051403e6..36a4cfcddb43c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -289,6 +289,22 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +======= +<<<<<<< HEAD +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#51] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index e3d76bfea8c2c..b050a6b78162d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -408,7 +408,23 @@ Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72] (73) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] +======= +<<<<<<< HEAD +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#75] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#74] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index aa9b899a9308c..fc3fffea0a40b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -289,6 +289,22 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +======= +<<<<<<< HEAD +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index e5e42f2be1366..f2fca6eefce59 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -1,74 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * SortMergeJoin Inner (49) - : : :- * Sort (37) - : : : +- Exchange (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- ReusedExchange (33) - : : +- * Sort (48) - : : +- Exchange (47) - : : +- * Project (46) - : : +- * BroadcastHashJoin Inner BuildRight (45) - : : :- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.inventory (38) - : : +- BroadcastExchange (44) - : : +- * Filter (43) - : : +- * ColumnarToRow (42) - : : +- Scan parquet default.warehouse (41) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * ColumnarToRow (52) - : +- Scan parquet default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * Project (62) - +- * Filter (61) - +- * ColumnarToRow (60) - +- Scan parquet default.catalog_returns (59) +TakeOrderedAndProject (68) ++- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * SortMergeJoin LeftOuter (63) + :- * Sort (56) + : +- * Project (55) + : +- * BroadcastHashJoin LeftOuter BuildRight (54) + : :- * Project (49) + : : +- * SortMergeJoin Inner (48) + : : :- * Sort (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * Project (32) + : : : : +- * SortMergeJoin Inner (31) + : : : : :- * Sort (25) + : : : : : +- Exchange (24) + : : : : : +- * Project (23) + : : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : : :- * Project (17) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : : :- * Project (10) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : : :- * Filter (3) + : : : : : : : : +- * ColumnarToRow (2) + : : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : : +- BroadcastExchange (8) + : : : : : : : +- * Project (7) + : : : : : : : +- * Filter (6) + : : : : : : : +- * ColumnarToRow (5) + : : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : : +- BroadcastExchange (15) + : : : : : : +- * Project (14) + : : : : : : +- * Filter (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- Scan parquet default.customer_demographics (11) + : : : : : +- BroadcastExchange (21) + : : : : : +- * Filter (20) + : : : : : +- * ColumnarToRow (19) + : : : : : +- Scan parquet default.date_dim (18) + : : : : +- * Sort (30) + : : : : +- Exchange (29) + : : : : +- * Filter (28) + : : : : +- * ColumnarToRow (27) + : : : : +- Scan parquet default.item (26) + : : : +- ReusedExchange (33) + : : +- * Sort (47) + : : +- Exchange (46) + : : +- * Project (45) + : : +- * BroadcastHashJoin Inner BuildRight (44) + : : :- * Filter (39) + : : : +- * ColumnarToRow (38) + : : : +- Scan parquet default.inventory (37) + : : +- BroadcastExchange (43) + : : +- * Filter (42) + : : +- * ColumnarToRow (41) + : : +- Scan parquet default.warehouse (40) + : +- BroadcastExchange (53) + : +- * Filter (52) + : +- * ColumnarToRow (51) + : +- Scan parquet default.promotion (50) + +- * Sort (62) + +- Exchange (61) + +- * Project (60) + +- * Filter (59) + +- * ColumnarToRow (58) + +- Scan parquet default.catalog_returns (57) (1) Scan parquet default.catalog_sales @@ -214,7 +212,7 @@ Join condition: None Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21] Input [8]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_sk#20, i_item_desc#21] -(33) ReusedExchange [Reuses operator id: 81] +(33) ReusedExchange [Reuses operator id: 79] Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] (34) BroadcastHashJoin [codegen id : 10] @@ -226,228 +224,260 @@ Join condition: (d_date#17 > date_add(d_date#24, 5)) Output [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, d_date#17, i_item_desc#21, d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -(36) Exchange -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] -Arguments: hashpartitioning(cs_item_sk#4, d_date_sk#26, 5), ENSURE_REQUIREMENTS, [id=#27] - -(37) Sort [codegen id : 11] +(36) Sort [codegen id : 10] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26] Arguments: [cs_item_sk#4 ASC NULLS FIRST, d_date_sk#26 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.inventory -Output [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(37) Scan parquet default.inventory +Output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#31), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] +(38) ColumnarToRow [codegen id : 12] +Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] -(40) Filter [codegen id : 13] -Input [4]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31] -Condition : ((isnotnull(inv_quantity_on_hand#30) AND isnotnull(inv_item_sk#28)) AND isnotnull(inv_warehouse_sk#29)) +(39) Filter [codegen id : 12] +Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] +Condition : ((isnotnull(inv_quantity_on_hand#29) AND isnotnull(inv_item_sk#27)) AND isnotnull(inv_warehouse_sk#28)) -(41) Scan parquet default.warehouse -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(40) Scan parquet default.warehouse +Output [2]: [w_warehouse_sk#31, w_warehouse_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(41) ColumnarToRow [codegen id : 11] +Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -(43) Filter [codegen id : 12] -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -Condition : isnotnull(w_warehouse_sk#32) +(42) Filter [codegen id : 11] +Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +Condition : isnotnull(w_warehouse_sk#31) +<<<<<<< HEAD (44) BroadcastExchange Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (45) BroadcastHashJoin [codegen id : 13] Left keys [1]: [inv_warehouse_sk#29] Right keys [1]: [w_warehouse_sk#32] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#34] +======= +(43) BroadcastExchange +Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +>>>>>>> 7190faa7a9 (update manifest files) + +(44) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [inv_warehouse_sk#28] +Right keys [1]: [w_warehouse_sk#31] +>>>>>>> 10c61d45bc (update manifest files) Join condition: None -(46) Project [codegen id : 13] -Output [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Input [6]: [inv_item_sk#28, inv_warehouse_sk#29, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(45) Project [codegen id : 12] +Output [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] +Input [6]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_sk#31, w_warehouse_name#32] -(47) Exchange -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: hashpartitioning(inv_item_sk#28, inv_date_sk#31, 5), ENSURE_REQUIREMENTS, [id=#35] +(46) Exchange +Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] +Arguments: hashpartitioning(inv_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#34] -(48) Sort [codegen id : 14] -Input [4]: [inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] -Arguments: [inv_item_sk#28 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +(47) Sort [codegen id : 13] +Input [4]: [inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] +Arguments: [inv_item_sk#27 ASC NULLS FIRST, inv_date_sk#30 ASC NULLS FIRST], false, 0 -(49) SortMergeJoin [codegen id : 16] +(48) SortMergeJoin [codegen id : 15] Left keys [2]: [cs_item_sk#4, d_date_sk#26] -Right keys [2]: [inv_item_sk#28, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#30 < cs_quantity#7) +Right keys [2]: [inv_item_sk#27, inv_date_sk#30] +Join condition: (inv_quantity_on_hand#29 < cs_quantity#7) -(50) Project [codegen id : 16] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#28, inv_quantity_on_hand#30, inv_date_sk#31, w_warehouse_name#33] +(49) Project [codegen id : 15] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Input [11]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, i_item_desc#21, d_week_seq#25, d_date_sk#26, inv_item_sk#27, inv_quantity_on_hand#29, inv_date_sk#30, w_warehouse_name#32] -(51) Scan parquet default.promotion -Output [1]: [p_promo_sk#36] +(50) Scan parquet default.promotion +Output [1]: [p_promo_sk#35] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 15] -Input [1]: [p_promo_sk#36] +(51) ColumnarToRow [codegen id : 14] +Input [1]: [p_promo_sk#35] -(53) Filter [codegen id : 15] -Input [1]: [p_promo_sk#36] -Condition : isnotnull(p_promo_sk#36) +(52) Filter [codegen id : 14] +Input [1]: [p_promo_sk#35] +Condition : isnotnull(p_promo_sk#35) +<<<<<<< HEAD (54) BroadcastExchange Input [1]: [p_promo_sk#36] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] - -(55) BroadcastHashJoin [codegen id : 16] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#37] +======= +(53) BroadcastExchange +Input [1]: [p_promo_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +(54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#36] +Right keys [1]: [p_promo_sk#35] Join condition: None -(56) Project [codegen id : 16] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, p_promo_sk#36] +(55) Project [codegen id : 15] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, p_promo_sk#35] -(57) Exchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [id=#38] - -(58) Sort [codegen id : 17] -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25] +(56) Sort [codegen id : 15] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(57) Scan parquet default.catalog_returns +Output [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(58) ColumnarToRow [codegen id : 16] +Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] -(61) Filter [codegen id : 18] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] -Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) +(59) Filter [codegen id : 16] +Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] +Condition : (isnotnull(cr_item_sk#37) AND isnotnull(cr_order_number#38)) -(62) Project [codegen id : 18] -Output [2]: [cr_item_sk#39, cr_order_number#40] -Input [3]: [cr_item_sk#39, cr_order_number#40, cr_returned_date_sk#41] +(60) Project [codegen id : 16] +Output [2]: [cr_item_sk#37, cr_order_number#38] +Input [3]: [cr_item_sk#37, cr_order_number#38, cr_returned_date_sk#39] -(63) Exchange -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: hashpartitioning(cr_item_sk#39, cr_order_number#40, 5), ENSURE_REQUIREMENTS, [id=#42] +(61) Exchange +Input [2]: [cr_item_sk#37, cr_order_number#38] +Arguments: hashpartitioning(cr_item_sk#37, 5), ENSURE_REQUIREMENTS, [id=#40] -(64) Sort [codegen id : 19] -Input [2]: [cr_item_sk#39, cr_order_number#40] -Arguments: [cr_item_sk#39 ASC NULLS FIRST, cr_order_number#40 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 17] +Input [2]: [cr_item_sk#37, cr_order_number#38] +Arguments: [cr_item_sk#37 ASC NULLS FIRST, cr_order_number#38 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 20] +(63) SortMergeJoin [codegen id : 18] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#39, cr_order_number#40] +Right keys [2]: [cr_item_sk#37, cr_order_number#38] Join condition: None -(66) Project [codegen id : 20] -Output [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#33, i_item_desc#21, d_week_seq#25, cr_item_sk#39, cr_order_number#40] +(64) Project [codegen id : 18] +Output [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#32, i_item_desc#21, d_week_seq#25, cr_item_sk#37, cr_order_number#38] -(67) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#33, i_item_desc#21, d_week_seq#25] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(65) HashAggregate [codegen id : 18] +Input [3]: [w_warehouse_name#32, i_item_desc#21, d_week_seq#25] +Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] +Aggregate Attributes [1]: [count#41] +Results [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42] -(68) Exchange -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#33, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#45] +(66) Exchange +Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42] +Arguments: hashpartitioning(i_item_desc#21, w_warehouse_name#32, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#43] -(69) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count#44] -Keys [3]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25] +(67) HashAggregate [codegen id : 19] +Input [4]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count#42] +Keys [3]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS no_promo#47, count(1)#46 AS promo#48, count(1)#46 AS total_cnt#49] +Aggregate Attributes [1]: [count(1)#44] +Results [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, count(1)#44 AS no_promo#45, count(1)#44 AS promo#46, count(1)#44 AS total_cnt#47] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] +(68) TakeOrderedAndProject +Input [6]: [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47] +Arguments: 100, [total_cnt#47 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#32 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#32, d_week_seq#25, no_promo#45, promo#46, total_cnt#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (81) -+- * Project (80) - +- * BroadcastHashJoin Inner BuildLeft (79) - :- BroadcastExchange (75) - : +- * Project (74) - : +- * Filter (73) - : +- * ColumnarToRow (72) - : +- Scan parquet default.date_dim (71) - +- * Filter (78) - +- * ColumnarToRow (77) - +- Scan parquet default.date_dim (76) - - -(71) Scan parquet default.date_dim -Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] +BroadcastExchange (79) ++- * Project (78) + +- * BroadcastHashJoin Inner BuildLeft (77) + :- BroadcastExchange (73) + : +- * Project (72) + : +- * Filter (71) + : +- * ColumnarToRow (70) + : +- Scan parquet default.date_dim (69) + +- * Filter (76) + +- * ColumnarToRow (75) + +- Scan parquet default.date_dim (74) + + +(69) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] +(70) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] -(73) Filter [codegen id : 1] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] -Condition : ((((isnotnull(d_year#50) AND (d_year#50 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) +(71) Filter [codegen id : 1] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] +Condition : ((((isnotnull(d_year#48) AND (d_year#48 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(74) Project [codegen id : 1] +(72) Project [codegen id : 1] Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#50] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] -(75) BroadcastExchange +(73) BroadcastExchange Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] - -(76) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#52] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#51] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) + +(74) Scan parquet default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(77) ColumnarToRow -Input [2]: [d_date_sk#26, d_week_seq#52] +(75) ColumnarToRow +Input [2]: [d_date_sk#26, d_week_seq#50] -(78) Filter -Input [2]: [d_date_sk#26, d_week_seq#52] -Condition : (isnotnull(d_week_seq#52) AND isnotnull(d_date_sk#26)) +(76) Filter +Input [2]: [d_date_sk#26, d_week_seq#50] +Condition : (isnotnull(d_week_seq#50) AND isnotnull(d_date_sk#26)) -(79) BroadcastHashJoin [codegen id : 2] +(77) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#52] +Right keys [1]: [d_week_seq#50] Join condition: None -(80) Project [codegen id : 2] +(78) Project [codegen id : 2] Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#52] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50] -(81) BroadcastExchange +(79) BroadcastExchange Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +>>>>>>> 7190faa7a9 (update manifest files) +>>>>>>> 10c61d45bc (update manifest files) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index e838025a71db8..d84393b2ff106 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -1,132 +1,126 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (21) + WholeStageCodegen (19) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (20) + WholeStageCodegen (18) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (15) Sort [cs_item_sk,cs_order_number] - InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (16) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] - InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] - InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (2) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + InputAdapter + WholeStageCodegen (10) + Sort [cs_item_sk,d_date_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #2 + WholeStageCodegen (4) + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #10 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #5 + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] + InputAdapter + ReusedExchange [d_date_sk,d_date,d_week_seq,d_date_sk] #3 + InputAdapter + WholeStageCodegen (13) + Sort [inv_item_sk,inv_date_sk] InputAdapter - WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] - InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 - WholeStageCodegen (13) - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + Exchange [inv_item_sk] #9 + WholeStageCodegen (12) + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (11) + Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (12) - Filter [w_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (15) - Filter [p_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.promotion [p_promo_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (14) + Filter [p_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (19) + WholeStageCodegen (17) Sort [cr_item_sk,cr_order_number] InputAdapter - Exchange [cr_item_sk,cr_order_number] #14 - WholeStageCodegen (18) + Exchange [cr_item_sk] #12 + WholeStageCodegen (16) Project [cr_item_sk,cr_order_number] Filter [cr_item_sk,cr_order_number] ColumnarToRow From 449db40eac7595fd5cadf39ffefd81c2a5324544 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Sat, 18 Sep 2021 13:55:46 -0700 Subject: [PATCH 09/29] bug fixes and more test cases --- .../plans/physical/partitioning.scala | 54 +++--- .../approved-plans-v1_4/q17.sf100/explain.txt | 4 +- .../q17.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q25.sf100/explain.txt | 4 +- .../q25.sf100/simplified.txt | 4 +- .../approved-plans-v1_4/q29.sf100/explain.txt | 4 +- .../q29.sf100/simplified.txt | 4 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 2 +- .../q24.sf100/simplified.txt | 2 +- .../spark/sql/execution/PlannerSuite.scala | 6 +- .../exchange/EnsureRequirementsSuite.scala | 156 +++++++++++++++++- 11 files changed, 196 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index f56bbb27472de..2436ab43adce8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -444,27 +444,31 @@ case class RangeShuffleSpec( override def isCompatibleWith(other: ShuffleSpec): Boolean = false - override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { - distribution.createPartitioning(numPartitions) - } + override def createPartitioning0(clustering: Seq[Expression]): Partitioning = + HashPartitioning(clustering, numPartitions) } case class HashShuffleSpec( partitioning: HashPartitioning, distribution: ClusteredDistribution) extends ShuffleSpec { - private lazy val matchingIndexes = indexMap(distribution.clustering, partitioning.expressions) + private lazy val hashKeyPositions = + createHashKeyPositions(distribution.clustering, partitioning.expressions) override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { case SinglePartitionShuffleSpec => partitioning.numPartitions == 1 - case HashShuffleSpec(partitioning, distribution) => - val expressions = partitioning.expressions + case HashShuffleSpec(otherPartitioning, otherDistribution) => // we need to check: // 1. both partitioning have the same number of expressions // 2. each corresponding expression in both partitioning is used in the same positions // of the corresponding distribution. - partitioning.expressions.length == expressions.length && - matchingIndexes == indexMap(distribution.clustering, expressions) + partitioning.expressions.length == otherPartitioning.expressions.length && { + val otherHashKeyPositions = createHashKeyPositions( + otherDistribution.clustering, otherPartitioning.expressions) + hashKeyPositions.zip(otherHashKeyPositions).forall { case (left, right) => + left.intersect(right).nonEmpty + } + } case ShuffleSpecCollection(specs) => specs.exists(isCompatibleWith) case _ => @@ -472,32 +476,26 @@ case class HashShuffleSpec( } override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { - val exprs = clustering - .zipWithIndex - .filter(x => matchingIndexes.keySet.contains(x._2)) - .map(_._1) + val exprs = hashKeyPositions.map(v => clustering(v.head)) HashPartitioning(exprs, partitioning.numPartitions) } override def numPartitions: Int = partitioning.numPartitions - // For each expression in the `expressions` that has occurrences in - // `clustering`, returns a mapping from its index in the partitioning to the - // indexes where it appears in the distribution. - // For instance, if `clustering` is [a, b] and `expressions` is [a, a, b], then the - // result mapping could be { 0 -> (0, 1), 1 -> (2) }. - private def indexMap( - clustering: Seq[Expression], - expressions: Seq[Expression]): mutable.Map[Int, mutable.BitSet] = { - val result = mutable.Map.empty[Int, mutable.BitSet] - val expressionToIndex = expressions.zipWithIndex.toMap - clustering.zipWithIndex.foreach { case (distKey, distKeyIdx) => - expressionToIndex.find { case (partKey, _) => partKey.semanticEquals(distKey) }.forall { - case (_, partIdx) => - result.getOrElseUpdate(partIdx, mutable.BitSet.empty).add(distKeyIdx) - } + /** + * Returns a sequence where each element is a set of positions of the key in `hashKeys` to its + * positions in `requiredClusterKeys`. For instance, if `requiredClusterKeys` is [a, b, b] and + * `hashKeys` is [a, b], the result will be [(0), (1, 2)]. + */ + private def createHashKeyPositions( + requiredClusterKeys: Seq[Expression], + hashKeys: Seq[Expression]): Seq[mutable.BitSet] = { + val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet] + requiredClusterKeys.zipWithIndex.foreach { case (distKey, distKeyPos) => + distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos) } - result + + hashKeys.map(k => distKeyToPos(k.canonicalized)) } } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 91ed657d870d5..baeffb7a77d91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -173,7 +173,7 @@ Input [6]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_qua (29) Exchange Input [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20] -Arguments: hashpartitioning(sr_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(sr_item_sk#17, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 10] Input [4]: [sr_item_sk#17, sr_customer_sk#18, sr_ticket_number#19, sr_return_quantity#20] @@ -217,7 +217,7 @@ Input [5]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27, cs_sold_date_ (39) Exchange Input [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27] -Arguments: hashpartitioning(cs_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(cs_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (40) Sort [codegen id : 14] Input [3]: [cs_bill_customer_sk#25, cs_item_sk#26, cs_quantity#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index 693bc90b720eb..b00c5da2ef7d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -61,7 +61,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (10) Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - Exchange [sr_customer_sk] #6 + Exchange [sr_item_sk] #6 WholeStageCodegen (9) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -83,7 +83,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (14) Sort [cs_bill_customer_sk,cs_item_sk] InputAdapter - Exchange [cs_bill_customer_sk] #8 + Exchange [cs_item_sk] #8 WholeStageCodegen (13) Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index b5bee8dc1213c..296cc496d6761 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -173,7 +173,7 @@ Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#2 (29) Exchange Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21] -Arguments: hashpartitioning(sr_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#25] +Arguments: hashpartitioning(sr_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#25] (30) Sort [codegen id : 10] Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_net_loss#21] @@ -217,7 +217,7 @@ Input [5]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28, cs_sold_dat (39) Exchange Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28] -Arguments: hashpartitioning(cs_bill_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] +Arguments: hashpartitioning(cs_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#31] (40) Sort [codegen id : 14] Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_net_profit#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index d0604e056b7cf..0b106ced5504d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -61,7 +61,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (10) Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - Exchange [sr_customer_sk] #6 + Exchange [sr_item_sk] #6 WholeStageCodegen (9) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -83,7 +83,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (14) Sort [cs_bill_customer_sk,cs_item_sk] InputAdapter - Exchange [cs_bill_customer_sk] #8 + Exchange [cs_item_sk] #8 WholeStageCodegen (13) Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index 7c12f3ca8d1e8..b32f5cd2d3b2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -173,7 +173,7 @@ Input [6]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_qua (29) Exchange Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] -Arguments: hashpartitioning(sr_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#25] +Arguments: hashpartitioning(sr_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#25] (30) Sort [codegen id : 10] Input [4]: [sr_item_sk#18, sr_customer_sk#19, sr_ticket_number#20, sr_return_quantity#21] @@ -217,7 +217,7 @@ Input [5]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28, cs_sold_date_ (39) Exchange Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28] -Arguments: hashpartitioning(cs_bill_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Arguments: hashpartitioning(cs_item_sk#27, 5), ENSURE_REQUIREMENTS, [id=#32] (40) Sort [codegen id : 14] Input [3]: [cs_bill_customer_sk#26, cs_item_sk#27, cs_quantity#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt index 4a2f65b44adc5..0db54fe759962 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt @@ -61,7 +61,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (10) Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - Exchange [sr_customer_sk] #6 + Exchange [sr_item_sk] #6 WholeStageCodegen (9) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -83,7 +83,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (14) Sort [cs_bill_customer_sk,cs_item_sk] InputAdapter - Exchange [cs_bill_customer_sk] #8 + Exchange [cs_item_sk] #8 WholeStageCodegen (13) Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index e10cef938901f..332a0b9220538 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -399,7 +399,7 @@ Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (70) Exchange Input [2]: [sr_item_sk#32, sr_ticket_number#33] -Arguments: hashpartitioning(sr_ticket_number#33, 5), ENSURE_REQUIREMENTS, [id=#53] +Arguments: hashpartitioning(sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#53] (71) Sort [codegen id : 10] Input [2]: [sr_item_sk#32, sr_ticket_number#33] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index a83743673a9c4..d12b734269651 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -52,7 +52,7 @@ WholeStageCodegen (12) WholeStageCodegen (10) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number] #14 + Exchange [sr_item_sk] #14 WholeStageCodegen (9) Project [sr_item_sk,sr_ticket_number] Filter [sr_ticket_number,sr_item_sk] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index df310cbaee455..11a453d831339 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -706,14 +706,14 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { outputPlan match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, - ShuffleExchangeExec(HashPartitioning(leftPartitioningExpressions, _), _, _), _), + DummySparkPlan(_, _, HashPartitioning(leftPartitioningExpressions, _), _, _), _), SortExec(_, _, ShuffleExchangeExec(HashPartitioning(rightPartitioningExpressions, _), _, _), _), _) => assert(leftKeys === smjExec.leftKeys) assert(rightKeys === smjExec.rightKeys) - assert(leftKeys === leftPartitioningExpressions) - assert(rightKeys === rightPartitioningExpressions) + assert(leftPartitioningExpressions == Seq(exprA, exprB, exprA)) + assert(rightPartitioningExpressions == Seq(exprA, exprC, exprA)) case _ => fail(outputPlan.toString) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index 10dc5ca4dc297..fd7ad1ceabf63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -29,6 +29,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { private val exprA = Literal(1) private val exprB = Literal(2) private val exprC = Literal(3) + private val exprD = Literal(4) private val EnsureRequirements = new EnsureRequirements() @@ -66,12 +67,12 @@ class EnsureRequirementsSuite extends SharedSparkSession { // Both sides are PartitioningCollection, but left side cannot be reordered to match // and it should fall back to the right side. val smjExec3 = SortMergeJoinExec( - exprA :: exprC :: Nil, exprB :: exprA :: Nil, Inner, None, plan1, plan1) + exprD :: exprC :: Nil, exprB :: exprA :: Nil, Inner, None, plan1, plan1) EnsureRequirements.apply(smjExec3) match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, ShuffleExchangeExec(_: HashPartitioning, _, _), _), SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), _) => - assert(leftKeys === Seq(exprC, exprA)) + assert(leftKeys === Seq(exprC, exprD)) assert(rightKeys === Seq(exprA, exprB)) case other => fail(other.toString) } @@ -136,7 +137,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("should have no shuffle when clustering keys are subset of join keys") { + test("SPARK-35703: should have no shuffle when clustering keys are subset of join keys") { val plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 5)) val plan2 = DummySparkPlan( @@ -164,4 +165,153 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } } + + test("SPARK-35703: compatibility with duplicated keys") { + var plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) + var plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprC :: Nil, 5)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB, exprB)) + assert(rightKeys === Seq(exprA, exprC, exprC)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprC :: exprA :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB, exprB)) + assert(rightKeys === Seq(exprA, exprC, exprC)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprC :: exprA :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB, exprB)) + assert(rightKeys === Seq(exprA, exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprC :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB, exprB)) + assert(rightKeys === Seq(exprA, exprC, exprC)) + case other => fail(other.toString) + } + } + + test("SPARK-35703: re-partition the other join side, basic case") { + var plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 10)) + var plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprD :: Nil, 5)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(p.expressions == Seq(exprC)) + case other => fail(other.toString) + } + + // rhs has more partitions so should be chosen + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprD :: Nil, 10)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(p.expressions == Seq(exprB)) + case other => fail(other.toString) + } + + // if both sides have the same # of partitions, should pick the first one from left + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprD :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(p.expressions == Seq(exprC)) + case other => fail(other.toString) + } + } + + test("SPARK-35703: re-partition the other join side, with duplicated keys") { + // with duplicate keys + var plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 10)) + var plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprC :: exprB :: Nil, 5)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprC :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB, exprB)) + assert(rightKeys === Seq(exprA, exprC, exprC)) + assert(p.expressions == Seq(exprA, exprC, exprA)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 10)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprC :: exprB :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprB :: Nil, exprA :: exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB, exprB)) + assert(rightKeys === Seq(exprA, exprC, exprD)) + assert(p.expressions == Seq(exprA, exprC, exprA)) + case other => fail(other.toString) + } + } } From 96f7e9ca0f223c7f28a8a61758aee2fac63f5d8a Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 29 Sep 2021 11:16:43 -0700 Subject: [PATCH 10/29] fix after rebase --- .../approved-plans-v1_4/q17.sf100/explain.txt | 10 ++ .../approved-plans-v1_4/q25.sf100/explain.txt | 10 ++ .../approved-plans-v1_4/q29.sf100/explain.txt | 16 ++- .../approved-plans-v1_4/q47.sf100/explain.txt | 4 + .../approved-plans-v1_4/q57.sf100/explain.txt | 4 + .../approved-plans-v1_4/q72.sf100/explain.txt | 20 +++- .../approved-plans-v2_7/q47.sf100/explain.txt | 4 + .../q51a.sf100/explain.txt | 4 + .../approved-plans-v2_7/q57.sf100/explain.txt | 4 + .../approved-plans-v2_7/q64/explain.txt | 101 ++++++++++++++++++ .../approved-plans-v2_7/q72.sf100/explain.txt | 20 +++- 11 files changed, 191 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index baeffb7a77d91..fa1e42309ee2e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -285,6 +285,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#89] (51) BroadcastExchange Input [1]: [d_date_sk#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] ======= <<<<<<< HEAD @@ -293,6 +294,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#90] +>>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#21 IN dynamicpruning#22 BroadcastExchange (56) @@ -320,6 +324,7 @@ Condition : (d_quarter_name#91 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Output [1]: [d_date_sk#23] Input [2]: [d_date_sk#23, d_quarter_name#91] +<<<<<<< HEAD <<<<<<< HEAD (59) BroadcastExchange Input [1]: [d_date_sk#24] @@ -333,6 +338,11 @@ Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +(56) BroadcastExchange +Input [1]: [d_date_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#92] +>>>>>>> 1afa00695f (fix after rebase) Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#28 IN dynamicpruning#22 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index 296cc496d6761..d6df26fa97a42 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -285,6 +285,7 @@ Input [3]: [d_date_sk#8, d_year#45, d_moy#46] (51) BroadcastExchange Input [1]: [d_date_sk#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] ======= <<<<<<< HEAD @@ -293,6 +294,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#47] +>>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 BroadcastExchange (56) @@ -320,6 +324,7 @@ Condition : (((((isnotnull(d_moy#49) AND isnotnull(d_year#48)) AND (d_moy#49 >= Output [1]: [d_date_sk#24] Input [3]: [d_date_sk#24, d_year#48, d_moy#49] +<<<<<<< HEAD <<<<<<< HEAD (59) BroadcastExchange Input [1]: [d_date_sk#25] @@ -333,6 +338,11 @@ Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +(56) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#50] +>>>>>>> 1afa00695f (fix after rebase) Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index b32f5cd2d3b2a..c7e12d2355d67 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -285,6 +285,7 @@ Input [3]: [d_date_sk#8, d_year#46, d_moy#47] (51) BroadcastExchange Input [1]: [d_date_sk#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] ======= <<<<<<< HEAD @@ -293,6 +294,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#48] +>>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 BroadcastExchange (56) @@ -320,6 +324,7 @@ Condition : (((((isnotnull(d_moy#50) AND isnotnull(d_year#49)) AND (d_moy#50 >= Output [1]: [d_date_sk#24] Input [3]: [d_date_sk#24, d_year#49, d_moy#50] +<<<<<<< HEAD <<<<<<< HEAD (59) BroadcastExchange Input [1]: [d_date_sk#25] @@ -339,10 +344,11 @@ Output [2]: [d_date_sk#33, d_year#54] ======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] ======= +======= +>>>>>>> 1afa00695f (fix after rebase) (56) BroadcastExchange Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] ->>>>>>> 7190faa7a9 (update manifest files) +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#51] Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#30 BroadcastExchange (61) @@ -371,6 +377,7 @@ Condition : (d_year#52 IN (1999,2000,2001) AND isnotnull(d_date_sk#31)) Output [1]: [d_date_sk#31] Input [2]: [d_date_sk#31, d_year#52] +<<<<<<< HEAD <<<<<<< HEAD (64) BroadcastExchange Input [1]: [d_date_sk#33] @@ -384,5 +391,10 @@ Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +(61) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] +>>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index c7a150e7621be..67adaf181701e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -290,6 +290,7 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] ======= <<<<<<< HEAD @@ -306,5 +307,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] +>>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index 1feb8125a359b..28fc8b3e16abe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -290,6 +290,7 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] ======= <<<<<<< HEAD @@ -306,5 +307,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#45] +>>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 836dfabaeaa36..a04fb4ee6e798 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -257,6 +257,7 @@ Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Condition : isnotnull(w_warehouse_sk#31) +<<<<<<< HEAD <<<<<<< HEAD (44) BroadcastExchange Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] @@ -269,10 +270,11 @@ Right keys [1]: [w_warehouse_sk#32] ======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#34] ======= +======= +>>>>>>> 1afa00695f (fix after rebase) (43) BroadcastExchange Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] ->>>>>>> 7190faa7a9 (update manifest files) +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#33] (44) BroadcastHashJoin [codegen id : 12] Left keys [1]: [inv_warehouse_sk#28] @@ -315,6 +317,7 @@ Input [1]: [p_promo_sk#35] Input [1]: [p_promo_sk#35] Condition : isnotnull(p_promo_sk#35) +<<<<<<< HEAD <<<<<<< HEAD (54) BroadcastExchange Input [1]: [p_promo_sk#36] @@ -328,6 +331,11 @@ Input [1]: [p_promo_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +(53) BroadcastExchange +Input [1]: [p_promo_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#36] +>>>>>>> 1afa00695f (fix after rebase) (54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_promo_sk#5] @@ -436,6 +444,7 @@ Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] (73) BroadcastExchange Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] ======= <<<<<<< HEAD @@ -444,6 +453,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#49] +>>>>>>> 1afa00695f (fix after rebase) (74) Scan parquet default.date_dim Output [2]: [d_date_sk#26, d_week_seq#50] @@ -471,6 +483,7 @@ Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50] (79) BroadcastExchange Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] ======= <<<<<<< HEAD @@ -479,5 +492,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#51] +>>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 36a4cfcddb43c..5a3ff837be4f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -290,6 +290,7 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] ======= <<<<<<< HEAD @@ -306,5 +307,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] +>>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index b050a6b78162d..e9a547decf70e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -409,6 +409,7 @@ Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72] (73) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] ======= <<<<<<< HEAD @@ -425,6 +426,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#73] +>>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index fc3fffea0a40b..d645e40a2bed2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -290,6 +290,7 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] ======= <<<<<<< HEAD @@ -306,5 +307,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#45] +>>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index cfee2290adff9..17dc817c8d864 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -688,9 +688,29 @@ Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#12 Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Condition : (((((((isnotnull(ss_item_sk#120) AND isnotnull(ss_ticket_number#127)) AND isnotnull(ss_store_sk#125)) AND isnotnull(ss_customer_sk#121)) AND isnotnull(ss_cdemo_sk#122)) AND isnotnull(ss_promo_sk#126)) AND isnotnull(ss_hdemo_sk#123)) AND isnotnull(ss_addr_sk#124)) +<<<<<<< HEAD (114) BroadcastExchange Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#133] +======= +<<<<<<< HEAD +<<<<<<< HEAD +(115) BroadcastExchange +Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#134] + +(116) Scan parquet default.store_returns +Output [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137] +======= +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false,false), [id=#134] +======= +======= +>>>>>>> 1afa00695f (fix after rebase) +(114) BroadcastExchange +Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false,false), [id=#133] +>>>>>>> 0b00377599 (fix after rebase) (115) Scan parquet default.store_returns Output [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] @@ -998,7 +1018,40 @@ Condition : ((isnotnull(d_year#43) AND (d_year#43 = 1999)) AND isnotnull(d_date_ (185) BroadcastExchange Input [2]: [d_date_sk#42, d_year#43] +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] +======= +<<<<<<< HEAD +<<<<<<< HEAD +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201] + +Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#132 IN dynamicpruning#133 +BroadcastExchange (191) ++- * Filter (190) + +- * ColumnarToRow (189) + +- Scan parquet default.date_dim (188) + + +======= +======= +>>>>>>> 61badbd9c1 (more) +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#201] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#207] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] +>>>>>>> 5527962502 (more) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#199] +>>>>>>> 1afa00695f (fix after rebase) +>>>>>>> 0b00377599 (fix after rebase) Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 BroadcastExchange (189) @@ -1014,6 +1067,33 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct +<<<<<<< HEAD +======= +<<<<<<< HEAD +<<<<<<< HEAD +<<<<<<< HEAD +(189) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#147, d_year#148] + +(190) Filter [codegen id : 1] +Input [2]: [d_date_sk#147, d_year#148] +Condition : ((isnotnull(d_year#148) AND (d_year#148 = 2000)) AND isnotnull(d_date_sk#147)) + +(191) BroadcastExchange +Input [2]: [d_date_sk#147, d_year#148] +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202] +======= +<<<<<<< HEAD +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#202] +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202] +======= +======= +>>>>>>> 5527962502 (more) +======= +>>>>>>> 1afa00695f (fix after rebase) +>>>>>>> 0b00377599 (fix after rebase) (187) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#146, d_year#147] @@ -1022,7 +1102,28 @@ Input [2]: [d_date_sk#146, d_year#147] Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (189) BroadcastExchange +<<<<<<< HEAD Input [2]: [d_date_sk#146, d_year#147] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] +======= +<<<<<<< HEAD +<<<<<<< HEAD +Input [2]: [d_date_sk#148, d_year#149] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#208] +>>>>>>> 7b99e32a26 (wip) +>>>>>>> 661766cab5 (wip) +<<<<<<< HEAD +>>>>>>> d784d1b800 (wip) +======= +======= +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] +>>>>>>> 5527962502 (more) +>>>>>>> 61badbd9c1 (more) +======= +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#200] +>>>>>>> 1afa00695f (fix after rebase) +>>>>>>> 0b00377599 (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index f2fca6eefce59..de63a32d192ed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -257,6 +257,7 @@ Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Condition : isnotnull(w_warehouse_sk#31) +<<<<<<< HEAD <<<<<<< HEAD (44) BroadcastExchange Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] @@ -269,10 +270,11 @@ Right keys [1]: [w_warehouse_sk#32] ======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#34] ======= +======= +>>>>>>> 1afa00695f (fix after rebase) (43) BroadcastExchange Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] ->>>>>>> 7190faa7a9 (update manifest files) +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#33] (44) BroadcastHashJoin [codegen id : 12] Left keys [1]: [inv_warehouse_sk#28] @@ -315,6 +317,7 @@ Input [1]: [p_promo_sk#35] Input [1]: [p_promo_sk#35] Condition : isnotnull(p_promo_sk#35) +<<<<<<< HEAD <<<<<<< HEAD (54) BroadcastExchange Input [1]: [p_promo_sk#36] @@ -328,6 +331,11 @@ Input [1]: [p_promo_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +(53) BroadcastExchange +Input [1]: [p_promo_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#36] +>>>>>>> 1afa00695f (fix after rebase) (54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_promo_sk#5] @@ -436,6 +444,7 @@ Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] (73) BroadcastExchange Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] ======= <<<<<<< HEAD @@ -444,6 +453,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#49] +>>>>>>> 1afa00695f (fix after rebase) (74) Scan parquet default.date_dim Output [2]: [d_date_sk#26, d_week_seq#50] @@ -471,6 +483,7 @@ Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50] (79) BroadcastExchange Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] ======= <<<<<<< HEAD @@ -479,5 +492,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] >>>>>>> 7190faa7a9 (update manifest files) >>>>>>> 10c61d45bc (update manifest files) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#51] +>>>>>>> 1afa00695f (fix after rebase) From c48c3788990b046a2d606fb21c3c3b144f5e12da Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 29 Sep 2021 12:25:53 -0700 Subject: [PATCH 11/29] add test and fixes --- .../plans/physical/partitioning.scala | 46 +- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 413 ++++++++++++++++++ .../exchange/EnsureRequirements.scala | 16 +- .../exchange/EnsureRequirementsSuite.scala | 166 ++++++- 4 files changed, 606 insertions(+), 35 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 2436ab43adce8..6c9bfb25fccc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -155,12 +155,10 @@ trait Partitioning { * shuffle shall be introduced. * * @param defaultNumPartitions the default number of partitions to use when creating a new - * partitioning from the spec + * partitioning using this spec * @param distribution the required distribution for this partitioning */ - final def createShuffleSpec( - defaultNumPartitions: Int, - distribution: Distribution): ShuffleSpec = + final def createShuffleSpec(defaultNumPartitions: Int, distribution: Distribution): ShuffleSpec = distribution match { case AllTuples => SinglePartitionShuffleSpec @@ -300,8 +298,13 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) override def createShuffleSpec0( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = { - val numPartitions = distribution.requiredNumPartitions.getOrElse(defaultNumPartitions) - RangeShuffleSpec(numPartitions, distribution) + // Since range partitioning is not even compatible with itself, we need to treat it especially. + // For instance, if left hand side is HashPartitioning(_, 100), while right hand side is + // RangePartitioning(_, 150), we should probably only shuffle the right hand side. + // However, if `spark.sql.shuffle.partitions` is 200, we should probably shuffle both sides. + // + // To achieve this, here we use the default number of partitions instead of `numPartitions`. + RangeShuffleSpec(defaultNumPartitions, distribution) } override protected def withNewChildrenInternal( @@ -351,10 +354,8 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) distribution: ClusteredDistribution): ShuffleSpec = { require(satisfies(distribution), "createShuffleSpec should only be called after satisfies " + "check is successful.") - val eligible = partitionings - .filter(_.satisfies(distribution)) - .map(_.createShuffleSpec(defaultNumPartitions, distribution)) - ShuffleSpecCollection(eligible) + val filtered = partitionings.filter(_.satisfies(distribution)) + ShuffleSpecCollection(filtered.map(_.createShuffleSpec(defaultNumPartitions, distribution))) } override def toString: String = { @@ -388,7 +389,7 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { * 1. Comparing with specs from other children of the operator and check if they are compatible. * When two specs are compatible, we can say their data are co-partitioned, and Spark will * potentially able to eliminate shuffle if necessary. - * 2. Creating a partitioning that can be used to re-partition another child, so that to make it + * 1. Creating a partitioning that can be used to re-partition another child, so that to make it * having a compatible partitioning as this node. */ trait ShuffleSpec { @@ -442,7 +443,11 @@ case class RangeShuffleSpec( numPartitions: Int, distribution: ClusteredDistribution) extends ShuffleSpec { - override def isCompatibleWith(other: ShuffleSpec): Boolean = false + override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { + case SinglePartitionShuffleSpec => numPartitions == 1 + case ShuffleSpecCollection(specs) => specs.exists(isCompatibleWith) + case _ => false + } override def createPartitioning0(clustering: Seq[Expression]): Partitioning = HashPartitioning(clustering, numPartitions) @@ -459,9 +464,11 @@ case class HashShuffleSpec( partitioning.numPartitions == 1 case HashShuffleSpec(otherPartitioning, otherDistribution) => // we need to check: - // 1. both partitioning have the same number of expressions - // 2. each corresponding expression in both partitioning is used in the same positions - // of the corresponding distribution. + // 1. both partitioning have the same number of partitions + // 2. both partitioning have the same number of expressions + // 3. each pair of expression from both has overlapping positions in their + // corresponding distributions. + partitioning.numPartitions == otherPartitioning.numPartitions && partitioning.expressions.length == otherPartitioning.expressions.length && { val otherHashKeyPositions = createHashKeyPositions( otherDistribution.clustering, otherPartitioning.expressions) @@ -501,13 +508,14 @@ case class HashShuffleSpec( case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec { override def isCompatibleWith(other: ShuffleSpec): Boolean = { - specs.exists(_.isCompatibleWith(this)) + specs.exists(_.isCompatibleWith(other)) } override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { - // choose the spec with the max number of partitions for better parallelism - specs.maxBy(_.numPartitions).createPartitioning0(clustering) + // as we only consider # of partitions as the cost now, it doesn't matter which one we choose + // since they should all have the same # of partitions. + specs.head.createPartitioning0(clustering) } - override def numPartitions: Int = specs.maxBy(_.numPartitions).numPartitions + override def numPartitions: Int = specs.head.numPartitions } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala new file mode 100644 index 0000000000000..1b5a0247d523b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala @@ -0,0 +1,413 @@ +/* + * 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.sql.catalyst + +import org.apache.spark.SparkFunSuite +/* Implicit conversions */ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.plans.physical._ + +class ShuffleSpecSuite extends SparkFunSuite { + protected def checkCompatible( + left: ShuffleSpec, + right: ShuffleSpec, + expected: Boolean): Unit = { + val actual = left.isCompatibleWith(right) + if (actual != expected) { + fail( + s""" + |== Left ShuffleSpec + |$left + |== Right ShuffleSpec + |$right + |== Does left is compatible with right? == + |Expected $expected but got $actual + |""".stripMargin + ) + } + } + + protected def checkCreatePartitioning( + spec: ShuffleSpec, + dist: Distribution, + expected: Partitioning): Unit = { + val actual = spec.createPartitioning(dist) + if (actual != expected) { + fail( + s""" + |== ShuffleSpec + |$spec + |== Distribution + |$dist + |== Result == + |Expected $expected but got $actual + |""".stripMargin + ) + } + } + + test("compatibility: HashShuffleSpec on both sides") { + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"d"), 10), ClusteredDistribution(Seq($"c", $"d"))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"c", $"c", $"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"c", $"a"), 10), + ClusteredDistribution(Seq($"a", $"c", $"c"))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"c", $"a"), 10), + ClusteredDistribution(Seq($"a", $"c", $"d"))), + expected = true + ) + + // negative cases + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"c"), 5), + ClusteredDistribution(Seq($"c", $"d"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + expected = false + ) + } + + test("compatibility: Only one side is HashShuffleSpec") { + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 1), + ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + expected = true + ) + + checkCompatible( + SinglePartitionShuffleSpec, + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 1), + ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) + + checkCompatible( + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + expected = true + ) + + checkCompatible( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"))))), + expected = false + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"))), + HashShuffleSpec(HashPartitioning(Seq($"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))))), + expected = true + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"))), + HashShuffleSpec(HashPartitioning(Seq($"c"), 10), + ClusteredDistribution(Seq($"c", $"d"))))), + expected = false + ) + } + + test("compatibility: other specs") { + checkCompatible( + SinglePartitionShuffleSpec, SinglePartitionShuffleSpec, expected = true + ) + + checkCompatible( + SinglePartitionShuffleSpec, + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) + + checkCompatible( + SinglePartitionShuffleSpec, + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), SinglePartitionShuffleSpec)), + expected = true + ) + + checkCompatible( + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) + + checkCompatible( + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + expected = false + ) + + checkCompatible( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + expected = true + ) + + checkCompatible( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), SinglePartitionShuffleSpec)), + expected = true + ) + + checkCompatible( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(1, ClusteredDistribution(Seq($"c", $"d"))))), + expected = false + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), SinglePartitionShuffleSpec)), + SinglePartitionShuffleSpec, + expected = true + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), SinglePartitionShuffleSpec)), + ShuffleSpecCollection(Seq( + SinglePartitionShuffleSpec, RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))))), + expected = true + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), SinglePartitionShuffleSpec)), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 1), + ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))))), + expected = true + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), SinglePartitionShuffleSpec)), + ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 2), + ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(2, ClusteredDistribution(Seq($"a", $"b"))))), + expected = false + ) + } + + test("createPartitioning: HashShuffleSpec") { + checkCreatePartitioning( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), + ClusteredDistribution(Seq($"c", $"d")), + HashPartitioning(Seq($"c"), 10) + ) + + checkCreatePartitioning( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + ClusteredDistribution(Seq($"a", $"c", $"c")), + HashPartitioning(Seq($"a", $"c", $"a"), 10) + ) + + checkCreatePartitioning( + HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + ClusteredDistribution(Seq($"a", $"c", $"c")), + HashPartitioning(Seq($"a", $"c", $"a"), 10) + ) + + checkCreatePartitioning( + HashShuffleSpec(HashPartitioning(Seq($"a", $"d"), 10), + ClusteredDistribution(Seq($"a", $"d", $"a", $"d"))), + ClusteredDistribution(Seq($"a", $"b", $"c", $"d")), + HashPartitioning(Seq($"a", $"b"), 10) + ) + } + + test("createPartitioning: other specs") { + checkCreatePartitioning(SinglePartitionShuffleSpec, + ClusteredDistribution(Seq($"a", $"b")), + SinglePartition + ) + + checkCreatePartitioning(SinglePartitionShuffleSpec, + ClusteredDistribution(Seq($"a", $"b")), + SinglePartition + ) + + checkCreatePartitioning(RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + ClusteredDistribution(Seq($"c", $"d")), + HashPartitioning(Seq($"c", $"d"), 10) + ) + + checkCreatePartitioning(ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))))), + ClusteredDistribution(Seq($"c", $"d")), + HashPartitioning(Seq($"c"), 10) + ) + } + + test("createPartitioning: AllTuples") { + Seq( + HashShuffleSpec(HashPartitioning(Seq($"a"), 1), ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), + ShuffleSpecCollection(Seq(SinglePartitionShuffleSpec, + RangeShuffleSpec(1, ClusteredDistribution(Seq($"a")))))).foreach { spec => + checkCreatePartitioning(spec, AllTuples, SinglePartition) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 8760e44a376a5..cc04b697f026c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -78,23 +78,29 @@ case class EnsureRequirements( if (childrenIndexes.length > 1) { val specs = childrenIndexes.map(i => i -> children(i).outputPartitioning.createShuffleSpec( - conf.numShufflePartitions, + requiredChildDistributions(i).requiredNumPartitions.getOrElse(conf.numShufflePartitions), requiredChildDistributions(i)) ).toMap - // one or more children have requirement on others for shuffle, so we need to check - // compatibility and come up a common partitioning for them + // Find out the shuffle spec that gives better parallelism. + // + // NOTE: this is not optimal for the case when there are more than 2 children. Consider: + // (10, 10, 11) + // it's better to pick 10 in this case since we only need to shuffle one side - we'd need to + // shuffle two sides if we pick 11. + // + // However this should be sufficient for now since in Spark nodes with multiple children + // always have exactly 2 children. val bestSpec = specs.values.maxBy(_.numPartitions) children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, _), idx) if !childrenIndexes.contains(idx) => child case ((child, dist), idx) => - // pick the best candidate from the requirements and use that to re-shuffle other - // children if necessary if (bestSpec.isCompatibleWith(specs(idx))) { child } else { + // Use the best spec to create a new partitioning to re-shuffle this child val newPartitioning = bestSpec.createPartitioning(dist) child match { case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(newPartitioning, c, so) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index fd7ad1ceabf63..032d0e422cef0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution.exchange -import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.{Ascending, Literal, SortOrder} import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection} +import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{DummySparkPlan, SortExec} import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.internal.SQLConf @@ -137,15 +137,27 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: should have no shuffle when clustering keys are subset of join keys") { + test("SPARK-35703: successful compatibility check with HashShuffleSpec") { val plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 5)) val plan2 = DummySparkPlan( outputPartitioning = HashPartitioning(exprB :: Nil, 5)) - val smjExec1 = SortMergeJoinExec( + var smjExec = SortMergeJoinExec( + exprA :: Nil, exprB :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA)) + assert(rightKeys === Seq(exprB)) + case other => fail(other.toString) + } + + // should also work if both partition keys are subset of their corresponding cluster keys + smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprB :: exprC :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec1) match { + EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => @@ -154,9 +166,9 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } - val smjExec2 = SortMergeJoinExec( + smjExec = SortMergeJoinExec( exprB :: exprA :: Nil, exprC :: exprB :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec2) match { + EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => @@ -166,7 +178,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: compatibility with duplicated keys") { + test("SPARK-35703: successful compatibility check with HashShuffleSpec and duplicate keys") { var plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) var plan2 = DummySparkPlan( @@ -228,7 +240,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: re-partition the other join side, basic case") { + test("SPARK-35703: incompatible & repartitioning with HashShuffleSpec") { var plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 10)) var plan2 = DummySparkPlan( @@ -280,8 +292,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: re-partition the other join side, with duplicated keys") { - // with duplicate keys + test("SPARK-35703: incompatible & repartitioning with HashShuffleSpec and duplicate keys") { var plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 10)) var plan2 = DummySparkPlan( @@ -314,4 +325,137 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } } + + test("SPARK-35703: successful compatibility check with other specs") { + var plan1 = DummySparkPlan(outputPartitioning = SinglePartition) + var plan2 = DummySparkPlan(outputPartitioning = SinglePartition) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, SinglePartition, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, SinglePartition, _, _), _), _) => + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = SinglePartition) + plan2 = DummySparkPlan(outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 1)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, SinglePartition, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprA, exprB), 10)))) + plan2 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprC, exprD), 10)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: PartitioningCollection, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + case other => fail(other.toString) + } + } + + test("SPARK-35703: incompatible & repartitioning with other specs") { + var plan1 = DummySparkPlan(outputPartitioning = SinglePartition) + var plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == conf.numShufflePartitions) + assert(right.numPartitions == conf.numShufflePartitions) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA, exprB), 10)) + plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == 10) + assert(right.numPartitions == 10) + assert(right.expressions == Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA), 1)) + plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == conf.numShufflePartitions) + assert(right.numPartitions == conf.numShufflePartitions) + assert(right.expressions == Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 1)) + plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprD, Ascending, sameOrderExpressions = Seq.empty)), 1)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == conf.numShufflePartitions) + assert(left.expressions == Seq(exprA, exprB)) + assert(right.numPartitions == conf.numShufflePartitions) + assert(right.expressions == Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprB), 10)))) + plan2 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprC), 10), HashPartitioning(Seq(exprD), 10)))) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil, + Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: PartitioningCollection, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == 10) + assert(right.numPartitions == 10) + assert(right.expressions == Seq(exprA)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprB), 10)))) + plan2 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprC), 20), HashPartitioning(Seq(exprD), 20)))) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil, + Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, right: PartitioningCollection, _, _), _), _) => + assert(left.numPartitions == 20) + assert(left.expressions == Seq(exprC)) + assert(right.numPartitions == 20) + case other => fail(other.toString) + } + } } From d0a5059ed601cdea516a5ac6f2384a7a9a856e2d Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 29 Sep 2021 22:30:53 -0700 Subject: [PATCH 12/29] revert pyspark change --- python/pyspark/sql/tests/test_pandas_cogrouped_map.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py index c01f60bb388d1..218e7cc77cf74 100644 --- a/python/pyspark/sql/tests/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py @@ -238,7 +238,6 @@ def right_assign_key(key, l, r): left.groupby("id") .cogroup(right.groupby("id")) .applyInPandas(right_assign_key, "id long, k int, v int, key long") - .sort(['id']) .toPandas() ) From ea8032a4c029c040515b82bf0f7036bdbaa48ff5 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 30 Sep 2021 17:23:24 -0700 Subject: [PATCH 13/29] fix the case on Exchange --- .../exchange/EnsureRequirements.scala | 43 +- .../exchange/EnsureRequirementsSuite.scala | 387 +++++++++++------- 2 files changed, 276 insertions(+), 154 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index cc04b697f026c..d07ddca18c562 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -91,7 +91,17 @@ case class EnsureRequirements( // // However this should be sufficient for now since in Spark nodes with multiple children // always have exactly 2 children. - val bestSpec = specs.values.maxBy(_.numPartitions) + // + // Also when choosing the spec, we should consider those children with no `Exchange` node + // first. For instance, if we have: + // A: (No_Exchange, 100) <---> B: (Exchange, 120) + // it's better to pick A and change B to (Exchange, 100) instead of picking B and insert a + // new shuffle for A. + val (shuffleSpecs, noShuffleSpecs) = specs.partition { + case (k, _) => children(k).isInstanceOf[ShuffleExchangeExec] + } + val bestSpec = (if (noShuffleSpecs.nonEmpty) noShuffleSpecs else specs) + .values.maxBy(_.numPartitions) children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, _), idx) if !childrenIndexes.contains(idx) => @@ -108,12 +118,10 @@ case class EnsureRequirements( } } } - } - val childrenNumPartitions = - childrenIndexes.map(children(_).outputPartitioning.numPartitions).toSet + val childrenNumPartitions = + childrenIndexes.map(children(_).outputPartitioning.numPartitions).toSet - if (childrenNumPartitions.size > 1) { // Get the number of partitions which is explicitly required by the distributions. val requiredNumPartitions = { val numPartitionsSet = childrenIndexes.flatMap { @@ -131,20 +139,21 @@ case class EnsureRequirements( val nonShuffleChildrenNumPartitions = childrenIndexes.map(children).filterNot(_.isInstanceOf[ShuffleExchangeExec]) .map(_.outputPartitioning.numPartitions) - val expectedChildrenNumPartitions = if (nonShuffleChildrenNumPartitions.nonEmpty) { - if (nonShuffleChildrenNumPartitions.length == childrenIndexes.length) { - // Here we pick the max number of partitions among these non-shuffle children. - nonShuffleChildrenNumPartitions.max + val expectedChildrenNumPartitions = + if (shuffleSpecs.nonEmpty && nonShuffleChildrenNumPartitions.nonEmpty) { + if (nonShuffleChildrenNumPartitions.length == childrenIndexes.length) { + // Here we pick the max number of partitions among these non-shuffle children. + nonShuffleChildrenNumPartitions.max + } else { + // Here we pick the max number of partitions among these non-shuffle children as the + // expected number of shuffle partitions. However, if it's smaller than + // `conf.numShufflePartitions`, we pick `conf.numShufflePartitions` as the + // expected number of shuffle partitions. + math.max(nonShuffleChildrenNumPartitions.max, conf.defaultNumShufflePartitions) + } } else { - // Here we pick the max number of partitions among these non-shuffle children as the - // expected number of shuffle partitions. However, if it's smaller than - // `conf.numShufflePartitions`, we pick `conf.numShufflePartitions` as the - // expected number of shuffle partitions. - math.max(nonShuffleChildrenNumPartitions.max, conf.defaultNumShufflePartitions) + childrenNumPartitions.max } - } else { - childrenNumPartitions.max - } val targetNumPartitions = requiredNumPartitions.getOrElse(expectedChildrenNumPartitions) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index 032d0e422cef0..d5dbc022d4602 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -241,54 +241,58 @@ class EnsureRequirementsSuite extends SharedSparkSession { } test("SPARK-35703: incompatible & repartitioning with HashShuffleSpec") { - var plan1 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprA :: Nil, 10)) - var plan2 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprD :: Nil, 5)) - var smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, - SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(p.expressions == Seq(exprC)) - case other => fail(other.toString) - } + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> 5.toString) { + var plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 10)) + var plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprD :: Nil, 5)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(p.expressions == Seq(exprC)) + case other => fail(other.toString) + } + + // rhs has more partitions so should be chosen + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprD :: Nil, 10)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(p.expressions == Seq(exprB)) + case other => fail(other.toString) + } + + // if both sides have the same # of partitions, should pick the first one from left + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprD :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(p.expressions == Seq(exprC)) + case other => fail(other.toString) + } - // rhs has more partitions so should be chosen - plan1 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprA :: Nil, 5)) - plan2 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprD :: Nil, 10)) - smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, - SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), - SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(p.expressions == Seq(exprB)) - case other => fail(other.toString) - } - // if both sides have the same # of partitions, should pick the first one from left - plan1 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprA :: Nil, 5)) - plan2 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprD :: Nil, 5)) - smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, - SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(p: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(p.expressions == Seq(exprC)) - case other => fail(other.toString) } } @@ -363,99 +367,208 @@ class EnsureRequirementsSuite extends SharedSparkSession { } test("SPARK-35703: incompatible & repartitioning with other specs") { - var plan1 = DummySparkPlan(outputPartitioning = SinglePartition) - var plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) - var smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == conf.numShufflePartitions) - assert(right.numPartitions == conf.numShufflePartitions) - case other => fail(other.toString) - } - - plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA, exprB), 10)) - plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 5)) - smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == 10) - assert(right.numPartitions == 10) - assert(right.expressions == Seq(exprC, exprD)) - case other => fail(other.toString) - } - - plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA), 1)) - plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) - smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == conf.numShufflePartitions) - assert(right.numPartitions == conf.numShufflePartitions) - assert(right.expressions == Seq(exprC, exprD)) - case other => fail(other.toString) - } - - plan1 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 1)) - plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprD, Ascending, sameOrderExpressions = Seq.empty)), 1)) - smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == conf.numShufflePartitions) - assert(left.expressions == Seq(exprA, exprB)) - assert(right.numPartitions == conf.numShufflePartitions) - assert(right.expressions == Seq(exprC, exprD)) - case other => fail(other.toString) - } - - plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( - HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprB), 10)))) - plan2 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( - HashPartitioning(Seq(exprC), 10), HashPartitioning(Seq(exprD), 10)))) - smjExec = SortMergeJoinExec( - exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil, - Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, DummySparkPlan(_, _, left: PartitioningCollection, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == 10) - assert(right.numPartitions == 10) - assert(right.expressions == Seq(exprA)) - case other => fail(other.toString) + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> 5.toString) { + var plan1 = DummySparkPlan(outputPartitioning = SinglePartition) + var plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == conf.numShufflePartitions) + assert(right.numPartitions == conf.numShufflePartitions) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA, exprB), 10)) + plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == 10) + assert(right.numPartitions == 10) + assert(right.expressions == Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA), 1)) + plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == conf.numShufflePartitions) + assert(right.numPartitions == conf.numShufflePartitions) + assert(right.expressions == Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 1)) + plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprD, Ascending, sameOrderExpressions = Seq.empty)), 1)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == conf.numShufflePartitions) + assert(left.expressions == Seq(exprA, exprB)) + assert(right.numPartitions == conf.numShufflePartitions) + assert(right.expressions == Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprB), 10)))) + plan2 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprC), 10), HashPartitioning(Seq(exprD), 10)))) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil, + Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: PartitioningCollection, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(left.numPartitions == 10) + assert(right.numPartitions == 10) + assert(right.expressions == Seq(exprA)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprB), 10)))) + plan2 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( + HashPartitioning(Seq(exprC), 20), HashPartitioning(Seq(exprD), 20)))) + smjExec = SortMergeJoinExec( + exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil, + Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, right: PartitioningCollection, _, _), _), _) => + assert(left.numPartitions == 20) + assert(left.expressions == Seq(exprC)) + assert(right.numPartitions == 20) + case other => fail(other.toString) + } } + } - plan1 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( - HashPartitioning(Seq(exprA), 10), HashPartitioning(Seq(exprB), 10)))) - plan2 = DummySparkPlan(outputPartitioning = PartitioningCollection(Seq( - HashPartitioning(Seq(exprC), 20), HashPartitioning(Seq(exprD), 20)))) - smjExec = SortMergeJoinExec( - exprA :: exprB :: exprC :: exprD :: Nil, exprA :: exprB :: exprC :: exprD :: Nil, - Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, DummySparkPlan(_, _, right: PartitioningCollection, _, _), _), _) => - assert(left.numPartitions == 20) - assert(left.expressions == Seq(exprC)) - assert(right.numPartitions == 20) - case other => fail(other.toString) + test("SPARK-35703: EnsureRequirements should respect spark.sql.shuffle.partitions") { + val defaultNumPartitions = 10 + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> defaultNumPartitions.toString) { + var plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) + var plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) + var smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 6)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == 6) + assert(right.numPartitions == 6) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan( + outputPartitioning = RangePartitioning( + Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 1)) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == defaultNumPartitions) + assert(right.numPartitions == defaultNumPartitions) + case other => fail(other.toString) + } + + plan1 = DummySparkPlan(outputPartitioning = SinglePartition) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == 5) + assert(right.numPartitions == 5) + case other => fail(other.toString) + } + + var plan3 = ShuffleExchangeExec( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 7), + child = DummySparkPlan()) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 6)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan3, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == defaultNumPartitions) + assert(right.numPartitions == defaultNumPartitions) + case other => fail(other.toString) + } + + plan3 = ShuffleExchangeExec( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 20), + child = DummySparkPlan()) + plan2 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 11)) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan3, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == 11) + assert(right.numPartitions == 11) + case other => fail(other.toString) + } } } } From 33a91bd9283f9f6fffc17a08362b9abc6d82ce86 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 1 Oct 2021 09:44:52 -0700 Subject: [PATCH 14/29] more comments and tests --- .../exchange/EnsureRequirements.scala | 47 ++++++++++--------- .../exchange/EnsureRequirementsSuite.scala | 20 ++++++++ 2 files changed, 45 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index d07ddca18c562..a5333d436c3db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -119,9 +119,6 @@ case class EnsureRequirements( } } - val childrenNumPartitions = - childrenIndexes.map(children(_).outputPartitioning.numPartitions).toSet - // Get the number of partitions which is explicitly required by the distributions. val requiredNumPartitions = { val numPartitionsSet = childrenIndexes.flatMap { @@ -137,10 +134,18 @@ case class EnsureRequirements( // 1. We should avoid shuffling these children. // 2. We should have a reasonable parallelism. val nonShuffleChildrenNumPartitions = - childrenIndexes.map(children).filterNot(_.isInstanceOf[ShuffleExchangeExec]) + childrenIndexes.map(children).filterNot(_.isInstanceOf[ShuffleExchangeExec]) .map(_.outputPartitioning.numPartitions) - val expectedChildrenNumPartitions = - if (shuffleSpecs.nonEmpty && nonShuffleChildrenNumPartitions.nonEmpty) { + + // If there was no `Exchange` in children before the above processing on shuffle specs, + // but new `Exchange` was added, then it could be two cases (assuming the node is binary): + // 1. one child A is shuffled matching the partitioning of the other child B, in which + // case we shouldn't consider `conf.numShufflePartitions` and shuffle B as well. See + // SPARK-32767 for the reasoning. + // 2. both children are re-shuffled, in which case they'll have the same number of + // partitions as guaranteed by the above procedure, so nothing to do here. + if (shuffleSpecs.nonEmpty && nonShuffleChildrenNumPartitions.nonEmpty) { + val expectedChildrenNumPartitions = if (nonShuffleChildrenNumPartitions.length == childrenIndexes.length) { // Here we pick the max number of partitions among these non-shuffle children. nonShuffleChildrenNumPartitions.max @@ -151,26 +156,24 @@ case class EnsureRequirements( // expected number of shuffle partitions. math.max(nonShuffleChildrenNumPartitions.max, conf.defaultNumShufflePartitions) } - } else { - childrenNumPartitions.max - } - val targetNumPartitions = requiredNumPartitions.getOrElse(expectedChildrenNumPartitions) + val targetNumPartitions = requiredNumPartitions.getOrElse(expectedChildrenNumPartitions) - children = children.zip(requiredChildDistributions).zipWithIndex.map { - case ((child, distribution), index) if childrenIndexes.contains(index) => - if (child.outputPartitioning.numPartitions == targetNumPartitions) { - child - } else { - val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) - child match { - // If child is an exchange, we replace it with a new one having defaultPartitioning. - case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) - case _ => ShuffleExchangeExec(defaultPartitioning, child) + children = children.zip(requiredChildDistributions).zipWithIndex.map { + case ((child, distribution), index) if childrenIndexes.contains(index) => + if (child.outputPartitioning.numPartitions == targetNumPartitions) { + child + } else { + val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) + child match { + // If child is an exchange, we replace it with a new one having defaultPartitioning. + case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) + case _ => ShuffleExchangeExec(defaultPartitioning, child) + } } - } - case ((child, _), _) => child + case ((child, _), _) => child + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index d5dbc022d4602..0ee0c6d5c0bcd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -569,6 +569,26 @@ class EnsureRequirementsSuite extends SharedSparkSession { assert(right.numPartitions == 11) case other => fail(other.toString) } + + // if both sides already have shuffle, we won't consider `conf.numShufflePartitions` + plan3 = ShuffleExchangeExec( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5), + child = DummySparkPlan()) + val plan4 = ShuffleExchangeExec( + outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 7), + child = DummySparkPlan()) + smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan3, plan4) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == 7) + assert(right.numPartitions == 7) + case other => fail(other.toString) + } } } } From 9a0d6c2f95bc15d4a2c7ead43540d76b6953d25a Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 14 Oct 2021 10:59:17 -0700 Subject: [PATCH 15/29] rebase --- .../approved-plans-v1_4/q17.sf100/explain.txt | 28 --------- .../approved-plans-v1_4/q25.sf100/explain.txt | 28 --------- .../approved-plans-v1_4/q29.sf100/explain.txt | 53 +---------------- .../approved-plans-v1_4/q47.sf100/explain.txt | 20 ------- .../approved-plans-v1_4/q57.sf100/explain.txt | 20 ------- .../approved-plans-v1_4/q72.sf100/explain.txt | 58 +------------------ .../approved-plans-v2_7/q47.sf100/explain.txt | 20 ------- .../q51a.sf100/explain.txt | 20 ------- .../approved-plans-v2_7/q57.sf100/explain.txt | 20 ------- .../approved-plans-v2_7/q64/explain.txt | 18 ++++++ .../approved-plans-v2_7/q72.sf100/explain.txt | 58 +------------------ 11 files changed, 21 insertions(+), 322 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index fa1e42309ee2e..16afa38901107 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -284,19 +284,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#89] (51) BroadcastExchange Input [1]: [d_date_sk#8] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#92] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#90] ->>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#21 IN dynamicpruning#22 BroadcastExchange (56) @@ -324,25 +312,9 @@ Condition : (d_quarter_name#91 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Output [1]: [d_date_sk#23] Input [2]: [d_date_sk#23, d_quarter_name#91] -<<<<<<< HEAD -<<<<<<< HEAD -(59) BroadcastExchange -Input [1]: [d_date_sk#24] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#94] -======= (56) BroadcastExchange Input [1]: [d_date_sk#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -(56) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#92] ->>>>>>> 1afa00695f (fix after rebase) Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#28 IN dynamicpruning#22 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index d6df26fa97a42..cbbf3da55739d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -284,19 +284,7 @@ Input [3]: [d_date_sk#8, d_year#45, d_moy#46] (51) BroadcastExchange Input [1]: [d_date_sk#8] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#49] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#47] ->>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 BroadcastExchange (56) @@ -324,25 +312,9 @@ Condition : (((((isnotnull(d_moy#49) AND isnotnull(d_year#48)) AND (d_moy#49 >= Output [1]: [d_date_sk#24] Input [3]: [d_date_sk#24, d_year#48, d_moy#49] -<<<<<<< HEAD -<<<<<<< HEAD -(59) BroadcastExchange -Input [1]: [d_date_sk#25] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#52] -======= (56) BroadcastExchange Input [1]: [d_date_sk#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -(56) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#50] ->>>>>>> 1afa00695f (fix after rebase) Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index c7e12d2355d67..e9857b76bc9e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -284,19 +284,7 @@ Input [3]: [d_date_sk#8, d_year#46, d_moy#47] (51) BroadcastExchange Input [1]: [d_date_sk#8] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#50] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#48] ->>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 23 Hosting Expression = sr_returned_date_sk#22 IN dynamicpruning#23 BroadcastExchange (56) @@ -324,31 +312,9 @@ Condition : (((((isnotnull(d_moy#50) AND isnotnull(d_year#49)) AND (d_moy#50 >= Output [1]: [d_date_sk#24] Input [3]: [d_date_sk#24, d_year#49, d_moy#50] -<<<<<<< HEAD -<<<<<<< HEAD -(59) BroadcastExchange -Input [1]: [d_date_sk#25] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] - -Subquery:3 Hosting operator id = 36 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#32 -BroadcastExchange (64) -+- * Project (63) - +- * Filter (62) - +- * ColumnarToRow (61) - +- Scan parquet default.date_dim (60) - - -(60) Scan parquet default.date_dim -Output [2]: [d_date_sk#33, d_year#54] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] -======= -======= ->>>>>>> 1afa00695f (fix after rebase) (56) BroadcastExchange Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#29 IN dynamicpruning#30 BroadcastExchange (61) @@ -360,7 +326,6 @@ BroadcastExchange (61) (57) Scan parquet default.date_dim Output [2]: [d_date_sk#31, d_year#52] ->>>>>>> 10c61d45bc (update manifest files) Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] @@ -377,24 +342,8 @@ Condition : (d_year#52 IN (1999,2000,2001) AND isnotnull(d_date_sk#31)) Output [1]: [d_date_sk#31] Input [2]: [d_date_sk#31, d_year#52] -<<<<<<< HEAD -<<<<<<< HEAD -(64) BroadcastExchange -Input [1]: [d_date_sk#33] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#55] -======= (61) BroadcastExchange Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -(61) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] ->>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index 67adaf181701e..44a956471b61e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -289,26 +289,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -<<<<<<< HEAD -<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] -======= -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#51] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] ->>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index 28fc8b3e16abe..ad356d44af668 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -289,26 +289,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -<<<<<<< HEAD -<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] -======= -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#45] ->>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index a04fb4ee6e798..d4ecd7a94c66a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -257,29 +257,13 @@ Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Condition : isnotnull(w_warehouse_sk#31) -<<<<<<< HEAD -<<<<<<< HEAD -(44) BroadcastExchange -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] - -(45) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [inv_warehouse_sk#29] -Right keys [1]: [w_warehouse_sk#32] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#34] -======= -======= ->>>>>>> 1afa00695f (fix after rebase) (43) BroadcastExchange Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (44) BroadcastHashJoin [codegen id : 12] Left keys [1]: [inv_warehouse_sk#28] Right keys [1]: [w_warehouse_sk#31] ->>>>>>> 10c61d45bc (update manifest files) Join condition: None (45) Project [codegen id : 12] @@ -317,25 +301,9 @@ Input [1]: [p_promo_sk#35] Input [1]: [p_promo_sk#35] Condition : isnotnull(p_promo_sk#35) -<<<<<<< HEAD -<<<<<<< HEAD -(54) BroadcastExchange -Input [1]: [p_promo_sk#36] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#37] -======= (53) BroadcastExchange Input [1]: [p_promo_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -(53) BroadcastExchange -Input [1]: [p_promo_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#36] ->>>>>>> 1afa00695f (fix after rebase) (54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_promo_sk#5] @@ -443,19 +411,7 @@ Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] (73) BroadcastExchange Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#51] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#49] ->>>>>>> 1afa00695f (fix after rebase) (74) Scan parquet default.date_dim Output [2]: [d_date_sk#26, d_week_seq#50] @@ -482,18 +438,6 @@ Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50] (79) BroadcastExchange Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#51] ->>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 5a3ff837be4f0..51b2f051403e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -289,26 +289,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -<<<<<<< HEAD -<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] -======= -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#51] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] ->>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index e9a547decf70e..e3d76bfea8c2c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -408,27 +408,7 @@ Input [3]: [d_date_sk#5, d_date#6, d_month_seq#72] (73) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -<<<<<<< HEAD -<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] -======= -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#75] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#74] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#73] ->>>>>>> 1afa00695f (fix after rebase) Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index d645e40a2bed2..aa9b899a9308c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -289,26 +289,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (53) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -<<<<<<< HEAD -<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] -======= -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#48] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#45] ->>>>>>> 1afa00695f (fix after rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 17dc817c8d864..06343101f5484 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -688,6 +688,7 @@ Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#12 Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Condition : (((((((isnotnull(ss_item_sk#120) AND isnotnull(ss_ticket_number#127)) AND isnotnull(ss_store_sk#125)) AND isnotnull(ss_customer_sk#121)) AND isnotnull(ss_cdemo_sk#122)) AND isnotnull(ss_promo_sk#126)) AND isnotnull(ss_hdemo_sk#123)) AND isnotnull(ss_addr_sk#124)) +<<<<<<< HEAD <<<<<<< HEAD (114) BroadcastExchange Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] @@ -711,6 +712,11 @@ Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false,false), [id=#133] >>>>>>> 0b00377599 (fix after rebase) +======= +(114) BroadcastExchange +Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#133] +>>>>>>> b5a1d077b4 (rebase) (115) Scan parquet default.store_returns Output [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] @@ -1019,6 +1025,7 @@ Condition : ((isnotnull(d_year#43) AND (d_year#43 = 1999)) AND isnotnull(d_date_ (185) BroadcastExchange Input [2]: [d_date_sk#42, d_year#43] <<<<<<< HEAD +<<<<<<< HEAD Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] ======= <<<<<<< HEAD @@ -1052,6 +1059,9 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#199] >>>>>>> 1afa00695f (fix after rebase) >>>>>>> 0b00377599 (fix after rebase) +======= +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] +>>>>>>> b5a1d077b4 (rebase) Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 BroadcastExchange (189) @@ -1067,6 +1077,7 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct +<<<<<<< HEAD <<<<<<< HEAD ======= <<<<<<< HEAD @@ -1094,6 +1105,8 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) ======= >>>>>>> 1afa00695f (fix after rebase) >>>>>>> 0b00377599 (fix after rebase) +======= +>>>>>>> b5a1d077b4 (rebase) (187) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#146, d_year#147] @@ -1103,6 +1116,7 @@ Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_dat (189) BroadcastExchange <<<<<<< HEAD +<<<<<<< HEAD Input [2]: [d_date_sk#146, d_year#147] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] ======= @@ -1125,5 +1139,9 @@ Input [2]: [d_date_sk#146, d_year#147] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#200] >>>>>>> 1afa00695f (fix after rebase) >>>>>>> 0b00377599 (fix after rebase) +======= +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] +>>>>>>> b5a1d077b4 (rebase) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index de63a32d192ed..42f7488ad66d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -257,29 +257,13 @@ Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] Condition : isnotnull(w_warehouse_sk#31) -<<<<<<< HEAD -<<<<<<< HEAD -(44) BroadcastExchange -Input [2]: [w_warehouse_sk#32, w_warehouse_name#33] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] - -(45) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [inv_warehouse_sk#29] -Right keys [1]: [w_warehouse_sk#32] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#34] -======= -======= ->>>>>>> 1afa00695f (fix after rebase) (43) BroadcastExchange Input [2]: [w_warehouse_sk#31, w_warehouse_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (44) BroadcastHashJoin [codegen id : 12] Left keys [1]: [inv_warehouse_sk#28] Right keys [1]: [w_warehouse_sk#31] ->>>>>>> 10c61d45bc (update manifest files) Join condition: None (45) Project [codegen id : 12] @@ -317,25 +301,9 @@ Input [1]: [p_promo_sk#35] Input [1]: [p_promo_sk#35] Condition : isnotnull(p_promo_sk#35) -<<<<<<< HEAD -<<<<<<< HEAD -(54) BroadcastExchange -Input [1]: [p_promo_sk#36] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#37] -======= (53) BroadcastExchange Input [1]: [p_promo_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -(53) BroadcastExchange -Input [1]: [p_promo_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#36] ->>>>>>> 1afa00695f (fix after rebase) (54) BroadcastHashJoin [codegen id : 15] Left keys [1]: [cs_promo_sk#5] @@ -443,19 +411,7 @@ Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#48] (73) BroadcastExchange Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#51] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#51] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#49] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false,false), [id=#49] ->>>>>>> 1afa00695f (fix after rebase) (74) Scan parquet default.date_dim Output [2]: [d_date_sk#26, d_week_seq#50] @@ -482,18 +438,6 @@ Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26, d_week_seq#50] (79) BroadcastExchange Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#26] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#53] -======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] ->>>>>>> 7190faa7a9 (update manifest files) ->>>>>>> 10c61d45bc (update manifest files) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false,false), [id=#51] ->>>>>>> 1afa00695f (fix after rebase) From 81b1a504945263f5ee1912aee69aca0f16a89d27 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Fri, 12 Nov 2021 11:16:41 -0800 Subject: [PATCH 16/29] address comments --- .../spark/sql/catalyst/plans/physical/partitioning.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 6c9bfb25fccc5..db9a850afe0b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -299,9 +299,10 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = { // Since range partitioning is not even compatible with itself, we need to treat it especially. - // For instance, if left hand side is HashPartitioning(_, 100), while right hand side is - // RangePartitioning(_, 150), we should probably only shuffle the right hand side. - // However, if `spark.sql.shuffle.partitions` is 200, we should probably shuffle both sides. + // For instance, suppose `spark.sql.shuffle.partitions` is 50, left hand side is + // HashPartitioning(_, 100), while right hand side is RangePartitioning(_, 150), we should + // probably only shuffle the right hand side. However, if `spark.sql.shuffle.partitions` is + // 200, we should probably shuffle both sides. // // To achieve this, here we use the default number of partitions instead of `numPartitions`. RangeShuffleSpec(defaultNumPartitions, distribution) From 708ee6f48e10a3ccbf47251f49a5a61a395a6373 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 18 Nov 2021 11:22:12 -0800 Subject: [PATCH 17/29] simplify the API and remove AllTuples case --- .../plans/physical/partitioning.scala | 60 ++++++------------- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 15 +---- .../exchange/EnsureRequirements.scala | 9 ++- 3 files changed, 29 insertions(+), 55 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index db9a850afe0b8..3a54479c64d0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -156,18 +156,12 @@ trait Partitioning { * * @param defaultNumPartitions the default number of partitions to use when creating a new * partitioning using this spec - * @param distribution the required distribution for this partitioning + * @param distribution the required clustered distribution for this partitioning */ - final def createShuffleSpec(defaultNumPartitions: Int, distribution: Distribution): ShuffleSpec = - distribution match { - case AllTuples => - SinglePartitionShuffleSpec - case clustered: ClusteredDistribution => - createShuffleSpec0(defaultNumPartitions, clustered) - case _ => - throw new IllegalStateException(s"Unexpected distribution: " + - s"${distribution.getClass.getSimpleName}") - } + def createShuffleSpec( + defaultNumPartitions: Int, + distribution: ClusteredDistribution): ShuffleSpec = + throw new IllegalStateException(s"Unexpected partitioning: ${getClass.getSimpleName}") /** * The actual method that defines whether this [[Partitioning]] can satisfy the given @@ -182,11 +176,6 @@ trait Partitioning { case AllTuples => numPartitions == 1 case _ => false } - - protected def createShuffleSpec0( - defaultNumPartitions: Int, - distribution: ClusteredDistribution): ShuffleSpec = - throw new IllegalStateException(s"Unexpected partitioning: ${getClass.getSimpleName}") } case class UnknownPartitioning(numPartitions: Int) extends Partitioning @@ -206,7 +195,7 @@ case object SinglePartition extends Partitioning { case _ => true } - override protected def createShuffleSpec0( + override def createShuffleSpec( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = SinglePartitionShuffleSpec } @@ -233,7 +222,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } - override def createShuffleSpec0( + override def createShuffleSpec( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = HashShuffleSpec(this, distribution) @@ -295,7 +284,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) } } - override def createShuffleSpec0( + override def createShuffleSpec( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = { // Since range partitioning is not even compatible with itself, we need to treat it especially. @@ -350,7 +339,7 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def satisfies0(required: Distribution): Boolean = partitionings.exists(_.satisfies(required)) - override def createShuffleSpec0( + override def createShuffleSpec( defaultNumPartitions: Int, distribution: ClusteredDistribution): ShuffleSpec = { require(satisfies(distribution), "createShuffleSpec should only be called after satisfies " + @@ -400,33 +389,22 @@ trait ShuffleSpec { def numPartitions: Int /** - * Returns true iff this spec is compatible with the other [[Partitioning]] and - * clustering expressions (e.g., from [[ClusteredDistribution]]). + * Returns true iff this spec is compatible with the provided shuffle spec. * * A true return value means that the data partitioning from this spec can be seen as - * co-partitioned with the `otherPartitioning`, and therefore no shuffle is required when - * joining the two sides. + * co-partitioned with the `other`, and therefore no shuffle is required when joining the two + * sides. */ def isCompatibleWith(other: ShuffleSpec): Boolean /** * Creates a partitioning that can be used to re-partitioned the other side with the given - * required distribution. + * clustering expressions. * * Note: this will only be called after `isCompatibleWith` returns true on the side where the * `clustering` is returned from. */ - final def createPartitioning(distribution: Distribution): Partitioning = distribution match { - case AllTuples => - SinglePartition - case ClusteredDistribution(clustering, _) => - createPartitioning0(clustering) - case _ => - throw new IllegalStateException("unexpected distribution: " + - s"${distribution.getClass.getSimpleName}") - } - - def createPartitioning0(clustering: Seq[Expression]): Partitioning + def createPartitioning(clustering: Seq[Expression]): Partitioning } case object SinglePartitionShuffleSpec extends ShuffleSpec { @@ -434,7 +412,7 @@ case object SinglePartitionShuffleSpec extends ShuffleSpec { other.numPartitions == numPartitions } - override def createPartitioning0(clustering: Seq[Expression]): Partitioning = + override def createPartitioning(clustering: Seq[Expression]): Partitioning = SinglePartition override def numPartitions: Int = 1 @@ -450,7 +428,7 @@ case class RangeShuffleSpec( case _ => false } - override def createPartitioning0(clustering: Seq[Expression]): Partitioning = + override def createPartitioning(clustering: Seq[Expression]): Partitioning = HashPartitioning(clustering, numPartitions) } @@ -483,7 +461,7 @@ case class HashShuffleSpec( false } - override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { val exprs = hashKeyPositions.map(v => clustering(v.head)) HashPartitioning(exprs, partitioning.numPartitions) } @@ -512,10 +490,10 @@ case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec { specs.exists(_.isCompatibleWith(other)) } - override def createPartitioning0(clustering: Seq[Expression]): Partitioning = { + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { // as we only consider # of partitions as the cost now, it doesn't matter which one we choose // since they should all have the same # of partitions. - specs.head.createPartitioning0(clustering) + specs.head.createPartitioning(clustering) } override def numPartitions: Int = specs.head.numPartitions diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala index 1b5a0247d523b..8a716305fbca8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala @@ -44,9 +44,9 @@ class ShuffleSpecSuite extends SparkFunSuite { protected def checkCreatePartitioning( spec: ShuffleSpec, - dist: Distribution, + dist: ClusteredDistribution, expected: Partitioning): Unit = { - val actual = spec.createPartitioning(dist) + val actual = spec.createPartitioning(dist.clustering) if (actual != expected) { fail( s""" @@ -399,15 +399,4 @@ class ShuffleSpecSuite extends SparkFunSuite { HashPartitioning(Seq($"c"), 10) ) } - - test("createPartitioning: AllTuples") { - Seq( - HashShuffleSpec(HashPartitioning(Seq($"a"), 1), ClusteredDistribution(Seq($"a", $"b"))), - SinglePartitionShuffleSpec, - RangeShuffleSpec(1, ClusteredDistribution(Seq($"a", $"b"))), - ShuffleSpecCollection(Seq(SinglePartitionShuffleSpec, - RangeShuffleSpec(1, ClusteredDistribution(Seq($"a")))))).foreach { spec => - checkCreatePartitioning(spec, AllTuples, SinglePartition) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index a5333d436c3db..2ac742b868485 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -70,16 +70,23 @@ case class EnsureRequirements( val childrenIndexes = requiredChildDistributions.zipWithIndex.filter { case (UnspecifiedDistribution, _) => false case (_: BroadcastDistribution, _) => false + case (AllTuples, _) => false case _ => true }.map(_._2) // If there are more than one children, we'll need to check partitioning & distribution of them // and see if extra shuffles are necessary. if (childrenIndexes.length > 1) { + childrenIndexes.map(requiredChildDistributions(_)).foreach { d => + if (!d.isInstanceOf[ClusteredDistribution]) { + throw new IllegalStateException(s"Expected ClusteredDistribution but found " + + s"${d.getClass.getSimpleName}") + } + } val specs = childrenIndexes.map(i => i -> children(i).outputPartitioning.createShuffleSpec( requiredChildDistributions(i).requiredNumPartitions.getOrElse(conf.numShufflePartitions), - requiredChildDistributions(i)) + requiredChildDistributions(i).asInstanceOf[ClusteredDistribution]) ).toMap // Find out the shuffle spec that gives better parallelism. From e7434197dc3b132c1c867e9adddd5812382b7c1b Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 18 Nov 2021 11:52:44 -0800 Subject: [PATCH 18/29] fix after rebase --- .../exchange/EnsureRequirements.scala | 3 +- .../approved-plans-v2_7/q64/explain.txt | 119 ------------------ 2 files changed, 2 insertions(+), 120 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 2ac742b868485..91b6a5298367b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -118,7 +118,8 @@ case class EnsureRequirements( child } else { // Use the best spec to create a new partitioning to re-shuffle this child - val newPartitioning = bestSpec.createPartitioning(dist) + val clustering = dist.asInstanceOf[ClusteredDistribution].clustering + val newPartitioning = bestSpec.createPartitioning(clustering) child match { case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(newPartitioning, c, so) case _ => ShuffleExchangeExec(newPartitioning, child) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 06343101f5484..cfee2290adff9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -688,35 +688,9 @@ Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#12 Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Condition : (((((((isnotnull(ss_item_sk#120) AND isnotnull(ss_ticket_number#127)) AND isnotnull(ss_store_sk#125)) AND isnotnull(ss_customer_sk#121)) AND isnotnull(ss_cdemo_sk#122)) AND isnotnull(ss_promo_sk#126)) AND isnotnull(ss_hdemo_sk#123)) AND isnotnull(ss_addr_sk#124)) -<<<<<<< HEAD -<<<<<<< HEAD (114) BroadcastExchange Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#133] -======= -<<<<<<< HEAD -<<<<<<< HEAD -(115) BroadcastExchange -Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_ticket_number#128, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#134] - -(116) Scan parquet default.store_returns -Output [3]: [sr_item_sk#135, sr_ticket_number#136, sr_returned_date_sk#137] -======= -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false,false), [id=#134] -======= -======= ->>>>>>> 1afa00695f (fix after rebase) -(114) BroadcastExchange -Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false,false), [id=#133] ->>>>>>> 0b00377599 (fix after rebase) -======= -(114) BroadcastExchange -Input [12]: [ss_item_sk#120, ss_customer_sk#121, ss_cdemo_sk#122, ss_hdemo_sk#123, ss_addr_sk#124, ss_store_sk#125, ss_promo_sk#126, ss_ticket_number#127, ss_wholesale_cost#128, ss_list_price#129, ss_coupon_amt#130, ss_sold_date_sk#131] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [id=#133] ->>>>>>> b5a1d077b4 (rebase) (115) Scan parquet default.store_returns Output [3]: [sr_item_sk#134, sr_ticket_number#135, sr_returned_date_sk#136] @@ -1024,44 +998,7 @@ Condition : ((isnotnull(d_year#43) AND (d_year#43 = 1999)) AND isnotnull(d_date_ (185) BroadcastExchange Input [2]: [d_date_sk#42, d_year#43] -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] -======= -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201] - -Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#132 IN dynamicpruning#133 -BroadcastExchange (191) -+- * Filter (190) - +- * ColumnarToRow (189) - +- Scan parquet default.date_dim (188) - - -======= -======= ->>>>>>> 61badbd9c1 (more) -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#201] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#207] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -======= Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] ->>>>>>> 5527962502 (more) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#199] ->>>>>>> 1afa00695f (fix after rebase) ->>>>>>> 0b00377599 (fix after rebase) -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#199] ->>>>>>> b5a1d077b4 (rebase) Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 BroadcastExchange (189) @@ -1077,36 +1014,6 @@ Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -<<<<<<< HEAD -<<<<<<< HEAD -======= -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -(189) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#147, d_year#148] - -(190) Filter [codegen id : 1] -Input [2]: [d_date_sk#147, d_year#148] -Condition : ((isnotnull(d_year#148) AND (d_year#148 = 2000)) AND isnotnull(d_date_sk#147)) - -(191) BroadcastExchange -Input [2]: [d_date_sk#147, d_year#148] -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202] -======= -<<<<<<< HEAD -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#202] -======= -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202] -======= -======= ->>>>>>> 5527962502 (more) -======= ->>>>>>> 1afa00695f (fix after rebase) ->>>>>>> 0b00377599 (fix after rebase) -======= ->>>>>>> b5a1d077b4 (rebase) (187) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#146, d_year#147] @@ -1115,33 +1022,7 @@ Input [2]: [d_date_sk#146, d_year#147] Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (189) BroadcastExchange -<<<<<<< HEAD -<<<<<<< HEAD -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] -======= -<<<<<<< HEAD -<<<<<<< HEAD -Input [2]: [d_date_sk#148, d_year#149] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#208] ->>>>>>> 7b99e32a26 (wip) ->>>>>>> 661766cab5 (wip) -<<<<<<< HEAD ->>>>>>> d784d1b800 (wip) -======= -======= -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] ->>>>>>> 5527962502 (more) ->>>>>>> 61badbd9c1 (more) -======= -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false,false), [id=#200] ->>>>>>> 1afa00695f (fix after rebase) ->>>>>>> 0b00377599 (fix after rebase) -======= Input [2]: [d_date_sk#146, d_year#147] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#200] ->>>>>>> b5a1d077b4 (rebase) From 24473ff5e45d0e8a67d3a95fe8bb99aecd5c965a Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Tue, 30 Nov 2021 16:52:23 -0800 Subject: [PATCH 19/29] trying a different approach of ordering specs --- .../plans/physical/partitioning.scala | 55 ++++---- .../exchange/EnsureRequirements.scala | 122 ++++++++---------- .../exchange/EnsureRequirementsSuite.scala | 51 ++++++-- 3 files changed, 116 insertions(+), 112 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 3a54479c64d0e..c2e57450de5ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -154,13 +154,9 @@ trait Partitioning { * used to decide whether this child is co-partitioned with others, therefore whether extra * shuffle shall be introduced. * - * @param defaultNumPartitions the default number of partitions to use when creating a new - * partitioning using this spec * @param distribution the required clustered distribution for this partitioning */ - def createShuffleSpec( - defaultNumPartitions: Int, - distribution: ClusteredDistribution): ShuffleSpec = + def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = throw new IllegalStateException(s"Unexpected partitioning: ${getClass.getSimpleName}") /** @@ -195,9 +191,8 @@ case object SinglePartition extends Partitioning { case _ => true } - override def createShuffleSpec( - defaultNumPartitions: Int, - distribution: ClusteredDistribution): ShuffleSpec = SinglePartitionShuffleSpec + override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = + SinglePartitionShuffleSpec } /** @@ -222,9 +217,8 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } - override def createShuffleSpec( - defaultNumPartitions: Int, - distribution: ClusteredDistribution): ShuffleSpec = HashShuffleSpec(this, distribution) + override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = + HashShuffleSpec(this, distribution) /** * Returns an expression that will produce a valid partition ID(i.e. non-negative and is less @@ -284,18 +278,8 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) } } - override def createShuffleSpec( - defaultNumPartitions: Int, - distribution: ClusteredDistribution): ShuffleSpec = { - // Since range partitioning is not even compatible with itself, we need to treat it especially. - // For instance, suppose `spark.sql.shuffle.partitions` is 50, left hand side is - // HashPartitioning(_, 100), while right hand side is RangePartitioning(_, 150), we should - // probably only shuffle the right hand side. However, if `spark.sql.shuffle.partitions` is - // 200, we should probably shuffle both sides. - // - // To achieve this, here we use the default number of partitions instead of `numPartitions`. - RangeShuffleSpec(defaultNumPartitions, distribution) - } + override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = + RangeShuffleSpec(this.numPartitions, distribution) override protected def withNewChildrenInternal( newChildren: IndexedSeq[Expression]): RangePartitioning = @@ -339,13 +323,11 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) override def satisfies0(required: Distribution): Boolean = partitionings.exists(_.satisfies(required)) - override def createShuffleSpec( - defaultNumPartitions: Int, - distribution: ClusteredDistribution): ShuffleSpec = { + override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = { require(satisfies(distribution), "createShuffleSpec should only be called after satisfies " + "check is successful.") val filtered = partitionings.filter(_.satisfies(distribution)) - ShuffleSpecCollection(filtered.map(_.createShuffleSpec(defaultNumPartitions, distribution))) + ShuffleSpecCollection(filtered.map(_.createShuffleSpec(distribution))) } override def toString: String = { @@ -397,12 +379,18 @@ trait ShuffleSpec { */ def isCompatibleWith(other: ShuffleSpec): Boolean + /** + * Whether this shuffle spec can be used to create partitionings for the other children. + */ + def canCreatePartitioning: Boolean + /** * Creates a partitioning that can be used to re-partitioned the other side with the given * clustering expressions. * - * Note: this will only be called after `isCompatibleWith` returns true on the side where the - * `clustering` is returned from. + * This will only be called when: + * - [[canCreatePartitioning]] returns true. + * - [[isCompatibleWith]] returns false on the side where the `clustering` is from. */ def createPartitioning(clustering: Seq[Expression]): Partitioning } @@ -412,6 +400,8 @@ case object SinglePartitionShuffleSpec extends ShuffleSpec { other.numPartitions == numPartitions } + override def canCreatePartitioning: Boolean = true + override def createPartitioning(clustering: Seq[Expression]): Partitioning = SinglePartition @@ -428,6 +418,8 @@ case class RangeShuffleSpec( case _ => false } + override def canCreatePartitioning: Boolean = false + override def createPartitioning(clustering: Seq[Expression]): Partitioning = HashPartitioning(clustering, numPartitions) } @@ -461,6 +453,8 @@ case class HashShuffleSpec( false } + override def canCreatePartitioning: Boolean = true + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { val exprs = hashKeyPositions.map(v => clustering(v.head)) HashPartitioning(exprs, partitioning.numPartitions) @@ -490,6 +484,9 @@ case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec { specs.exists(_.isCompatibleWith(other)) } + override def canCreatePartitioning: Boolean = + specs.forall(_.canCreatePartitioning) + override def createPartitioning(clustering: Seq[Expression]): Partitioning = { // as we only consider # of partitions as the cost now, it doesn't matter which one we choose // since they should all have the same # of partitions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 91b6a5298367b..9bdece45615ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -85,7 +85,6 @@ case class EnsureRequirements( } val specs = childrenIndexes.map(i => i -> children(i).outputPartitioning.createShuffleSpec( - requiredChildDistributions(i).requiredNumPartitions.getOrElse(conf.numShufflePartitions), requiredChildDistributions(i).asInstanceOf[ClusteredDistribution]) ).toMap @@ -98,91 +97,72 @@ case class EnsureRequirements( // // However this should be sufficient for now since in Spark nodes with multiple children // always have exactly 2 children. + + // Whether we should consider `spark.sql.shuffle.partitions` and ensure enough parallelism + // during the shuffle. To achieve a good trade-off between parallelism and shuffle cost, we + // only consider the minimum parallelism if: + // 1. Some child can't create partitioning, i.e., it needs to be shuffled. + // 2. Some child already needs to be shuffled with `ShuffleExchangeExec` being present. + // In either of the above cases, we'll apply `spark.sql.shuffle.partitions` in case there + // is not enough parallelism. // - // Also when choosing the spec, we should consider those children with no `Exchange` node - // first. For instance, if we have: - // A: (No_Exchange, 100) <---> B: (Exchange, 120) - // it's better to pick A and change B to (Exchange, 100) instead of picking B and insert a - // new shuffle for A. - val (shuffleSpecs, noShuffleSpecs) = specs.partition { - case (k, _) => children(k).isInstanceOf[ShuffleExchangeExec] + // On the other hand, if we have: + // HashPartitioning(5) <-> HashPartitioning(6) + // while `spark.sql.shuffle.partitions` is 10, we'll only re-shuffle the left side and make it + // HashPartitioning(6). + val shouldRespectMinPartitions = specs.exists(p => + !p._2.canCreatePartitioning || children(p._1).isInstanceOf[ShuffleExchangeExec] + ) + // Choose all the specs that can be used to shuffle other children + val candidateSpecs = specs + .filter(_._2.canCreatePartitioning) + .filter(p => shouldRespectMinPartitions && + children(p._1).outputPartitioning.numPartitions >= conf.defaultNumShufflePartitions) + val bestSpec = if (candidateSpecs.isEmpty) { + None + } else { + // When choosing specs, we should consider those children with no `Exchange` node + // first. For instance, if we have: + // A: (No_Exchange, 100) <---> B: (Exchange, 120) + // it's better to pick A and change B to (Exchange, 100) instead of picking B and insert a + // new shuffle for A. + val candidateSpecsWithoutShuffle = candidateSpecs.filter { case (k, _) => + !children(k).isInstanceOf[ShuffleExchangeExec] + } + val specs = if (candidateSpecsWithoutShuffle.nonEmpty) { + candidateSpecsWithoutShuffle + } else { + candidateSpecs + } + // Pick the spec with the best parallelism + Some(specs.values.maxBy(_.numPartitions)) } - val bestSpec = (if (noShuffleSpecs.nonEmpty) noShuffleSpecs else specs) - .values.maxBy(_.numPartitions) children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, _), idx) if !childrenIndexes.contains(idx) => child case ((child, dist), idx) => - if (bestSpec.isCompatibleWith(specs(idx))) { + if (bestSpec.isDefined && bestSpec.get.isCompatibleWith(specs(idx))) { child } else { - // Use the best spec to create a new partitioning to re-shuffle this child - val clustering = dist.asInstanceOf[ClusteredDistribution].clustering - val newPartitioning = bestSpec.createPartitioning(clustering) + val newPartitioning = if (bestSpec.isDefined) { + // Use the best spec to create a new partitioning to re-shuffle this child + val clustering = dist.asInstanceOf[ClusteredDistribution].clustering + bestSpec.get.createPartitioning(clustering) + } else { + // No best spec available, so we create default partitioning from the required + // distribution + val numPartitions = dist.requiredNumPartitions + .getOrElse(conf.numShufflePartitions) + dist.createPartitioning(numPartitions) + } + child match { case ShuffleExchangeExec(_, c, so) => ShuffleExchangeExec(newPartitioning, c, so) case _ => ShuffleExchangeExec(newPartitioning, child) } } } - - // Get the number of partitions which is explicitly required by the distributions. - val requiredNumPartitions = { - val numPartitionsSet = childrenIndexes.flatMap { - index => requiredChildDistributions(index).requiredNumPartitions - }.toSet - assert(numPartitionsSet.size <= 1, - s"$requiredChildDistributions have incompatible requirements of the number of partitions") - numPartitionsSet.headOption - } - - // If there are non-shuffle children that satisfy the required distribution, we have - // some tradeoffs when picking the expected number of shuffle partitions: - // 1. We should avoid shuffling these children. - // 2. We should have a reasonable parallelism. - val nonShuffleChildrenNumPartitions = - childrenIndexes.map(children).filterNot(_.isInstanceOf[ShuffleExchangeExec]) - .map(_.outputPartitioning.numPartitions) - - // If there was no `Exchange` in children before the above processing on shuffle specs, - // but new `Exchange` was added, then it could be two cases (assuming the node is binary): - // 1. one child A is shuffled matching the partitioning of the other child B, in which - // case we shouldn't consider `conf.numShufflePartitions` and shuffle B as well. See - // SPARK-32767 for the reasoning. - // 2. both children are re-shuffled, in which case they'll have the same number of - // partitions as guaranteed by the above procedure, so nothing to do here. - if (shuffleSpecs.nonEmpty && nonShuffleChildrenNumPartitions.nonEmpty) { - val expectedChildrenNumPartitions = - if (nonShuffleChildrenNumPartitions.length == childrenIndexes.length) { - // Here we pick the max number of partitions among these non-shuffle children. - nonShuffleChildrenNumPartitions.max - } else { - // Here we pick the max number of partitions among these non-shuffle children as the - // expected number of shuffle partitions. However, if it's smaller than - // `conf.numShufflePartitions`, we pick `conf.numShufflePartitions` as the - // expected number of shuffle partitions. - math.max(nonShuffleChildrenNumPartitions.max, conf.defaultNumShufflePartitions) - } - - val targetNumPartitions = requiredNumPartitions.getOrElse(expectedChildrenNumPartitions) - - children = children.zip(requiredChildDistributions).zipWithIndex.map { - case ((child, distribution), index) if childrenIndexes.contains(index) => - if (child.outputPartitioning.numPartitions == targetNumPartitions) { - child - } else { - val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) - child match { - // If child is an exchange, we replace it with a new one having defaultPartitioning. - case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) - case _ => ShuffleExchangeExec(defaultPartitioning, child) - } - } - - case ((child, _), _) => child - } - } } // Now that we've performed any necessary shuffles, add sorts to guarantee output orderings: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index 0ee0c6d5c0bcd..c13581cbc1545 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Literal, SortOrder} import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{DummySparkPlan, SortExec} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -468,9 +469,9 @@ class EnsureRequirementsSuite extends SharedSparkSession { test("SPARK-35703: EnsureRequirements should respect spark.sql.shuffle.partitions") { val defaultNumPartitions = 10 withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> defaultNumPartitions.toString) { - var plan1 = DummySparkPlan( + var plan1: SparkPlan = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) - var plan2 = DummySparkPlan( + var plan2: SparkPlan = DummySparkPlan( outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) var smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) @@ -534,13 +535,13 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } - var plan3 = ShuffleExchangeExec( + plan1 = ShuffleExchangeExec( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 7), child = DummySparkPlan()) plan2 = DummySparkPlan( outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 6)) smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan3, plan2) + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), @@ -552,13 +553,13 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } - plan3 = ShuffleExchangeExec( + plan1 = ShuffleExchangeExec( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 20), child = DummySparkPlan()) plan2 = DummySparkPlan( outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 11)) smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan3, plan2) + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), @@ -570,25 +571,51 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } - // if both sides already have shuffle, we won't consider `conf.numShufflePartitions` - plan3 = ShuffleExchangeExec( + // if both sides already have shuffle, we should consider `conf.numShufflePartitions` + plan1 = ShuffleExchangeExec( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5), child = DummySparkPlan()) - val plan4 = ShuffleExchangeExec( + plan2 = ShuffleExchangeExec( outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 7), child = DummySparkPlan()) smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan3, plan4) + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(leftKeys, rightKeys, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => assert(leftKeys === Seq(exprA, exprB)) assert(rightKeys === Seq(exprC, exprD)) - assert(left.numPartitions == 7) - assert(right.numPartitions == 7) + assert(left.numPartitions == 10) + assert(right.numPartitions == 10) case other => fail(other.toString) } } } + + test("Respect spark.sql.shuffle.partitions with AQE") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> 8.toString, + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> 10.toString) { + Seq(false).foreach { enable => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> s"$enable") { + val plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 9)) + val plan2 = DummySparkPlan( + outputPartitioning = UnknownPartitioning(8)) + val smjExec = SortMergeJoinExec( + exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) + EnsureRequirements.apply(smjExec) match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), + SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => + assert(leftKeys === Seq(exprA, exprB)) + assert(rightKeys === Seq(exprC, exprD)) + assert(left.numPartitions == 9) + assert(right.numPartitions == 9) + case other => fail(other.toString) + } + } + } + } + } } From 17e7b3ff00ba19bf6d5cc506f3259a5953f9320f Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 1 Dec 2021 10:35:41 -0800 Subject: [PATCH 20/29] remove require check --- .../apache/spark/sql/catalyst/plans/physical/partitioning.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index c2e57450de5ae..0e2dd88b3291a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -324,8 +324,6 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) partitionings.exists(_.satisfies(required)) override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = { - require(satisfies(distribution), "createShuffleSpec should only be called after satisfies " + - "check is successful.") val filtered = partitionings.filter(_.satisfies(distribution)) ShuffleSpecCollection(filtered.map(_.createShuffleSpec(distribution))) } From bc41e1ced51e7d968435bdb5ecc85f68609f5069 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 1 Dec 2021 16:58:29 -0800 Subject: [PATCH 21/29] fix --- .../spark/sql/execution/exchange/EnsureRequirements.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 9bdece45615ab..f6080fe576905 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -110,13 +110,13 @@ case class EnsureRequirements( // HashPartitioning(5) <-> HashPartitioning(6) // while `spark.sql.shuffle.partitions` is 10, we'll only re-shuffle the left side and make it // HashPartitioning(6). - val shouldRespectMinPartitions = specs.exists(p => - !p._2.canCreatePartitioning || children(p._1).isInstanceOf[ShuffleExchangeExec] + val canIgnoreMinPartitions = specs.forall(p => + p._2.canCreatePartitioning && !children(p._1).isInstanceOf[ShuffleExchangeExec] ) // Choose all the specs that can be used to shuffle other children val candidateSpecs = specs .filter(_._2.canCreatePartitioning) - .filter(p => shouldRespectMinPartitions && + .filter(p => canIgnoreMinPartitions || children(p._1).outputPartitioning.numPartitions >= conf.defaultNumShufflePartitions) val bestSpec = if (candidateSpecs.isEmpty) { None From fc843c445cb9a204997bb5d929cc62983a28e9ed Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Mon, 6 Dec 2021 10:38:19 -0800 Subject: [PATCH 22/29] address comments --- .../plans/physical/partitioning.scala | 18 ++++++------- .../exchange/EnsureRequirements.scala | 25 ++++++++----------- 2 files changed, 17 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 0e2dd88b3291a..1ebdc5640800d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -380,7 +380,7 @@ trait ShuffleSpec { /** * Whether this shuffle spec can be used to create partitionings for the other children. */ - def canCreatePartitioning: Boolean + def canCreatePartitioning: Boolean = false /** * Creates a partitioning that can be used to re-partitioned the other side with the given @@ -390,7 +390,9 @@ trait ShuffleSpec { * - [[canCreatePartitioning]] returns true. * - [[isCompatibleWith]] returns false on the side where the `clustering` is from. */ - def createPartitioning(clustering: Seq[Expression]): Partitioning + def createPartitioning(clustering: Seq[Expression]): Partitioning = + throw new UnsupportedOperationException("Operation unsupported for " + + s"${getClass.getCanonicalName}") } case object SinglePartitionShuffleSpec extends ShuffleSpec { @@ -415,23 +417,18 @@ case class RangeShuffleSpec( case ShuffleSpecCollection(specs) => specs.exists(isCompatibleWith) case _ => false } - - override def canCreatePartitioning: Boolean = false - - override def createPartitioning(clustering: Seq[Expression]): Partitioning = - HashPartitioning(clustering, numPartitions) } case class HashShuffleSpec( partitioning: HashPartitioning, distribution: ClusteredDistribution) extends ShuffleSpec { - private lazy val hashKeyPositions = + lazy val hashKeyPositions = createHashKeyPositions(distribution.clustering, partitioning.expressions) override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { case SinglePartitionShuffleSpec => partitioning.numPartitions == 1 - case HashShuffleSpec(otherPartitioning, otherDistribution) => + case otherHashSpec @ HashShuffleSpec(otherPartitioning, otherDistribution) => // we need to check: // 1. both partitioning have the same number of partitions // 2. both partitioning have the same number of expressions @@ -439,8 +436,7 @@ case class HashShuffleSpec( // corresponding distributions. partitioning.numPartitions == otherPartitioning.numPartitions && partitioning.expressions.length == otherPartitioning.expressions.length && { - val otherHashKeyPositions = createHashKeyPositions( - otherDistribution.clustering, otherPartitioning.expressions) + val otherHashKeyPositions = otherHashSpec.hashKeyPositions hashKeyPositions.zip(otherHashKeyPositions).forall { case (left, right) => left.intersect(right).nonEmpty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index f6080fe576905..577a5a67b1593 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -68,25 +68,20 @@ case class EnsureRequirements( // Get the indexes of children which have specified distribution requirements and need to have // same number of partitions. val childrenIndexes = requiredChildDistributions.zipWithIndex.filter { - case (UnspecifiedDistribution, _) => false - case (_: BroadcastDistribution, _) => false - case (AllTuples, _) => false - case _ => true + case (_: ClusteredDistribution, _) => true + case _ => false }.map(_._2) // If there are more than one children, we'll need to check partitioning & distribution of them // and see if extra shuffles are necessary. if (childrenIndexes.length > 1) { - childrenIndexes.map(requiredChildDistributions(_)).foreach { d => - if (!d.isInstanceOf[ClusteredDistribution]) { - throw new IllegalStateException(s"Expected ClusteredDistribution but found " + - s"${d.getClass.getSimpleName}") - } - } - val specs = childrenIndexes.map(i => + val specs = childrenIndexes.map(i => { + val requiredDist = requiredChildDistributions(i) + assert(requiredDist.isInstanceOf[ClusteredDistribution], + s"Expected ClusteredDistribution but found ${requiredDist.getClass.getSimpleName}") i -> children(i).outputPartitioning.createShuffleSpec( - requiredChildDistributions(i).asInstanceOf[ClusteredDistribution]) - ).toMap + requiredDist.asInstanceOf[ClusteredDistribution]) + }).toMap // Find out the shuffle spec that gives better parallelism. // @@ -129,13 +124,13 @@ case class EnsureRequirements( val candidateSpecsWithoutShuffle = candidateSpecs.filter { case (k, _) => !children(k).isInstanceOf[ShuffleExchangeExec] } - val specs = if (candidateSpecsWithoutShuffle.nonEmpty) { + val finalCandidateSpecs = if (candidateSpecsWithoutShuffle.nonEmpty) { candidateSpecsWithoutShuffle } else { candidateSpecs } // Pick the spec with the best parallelism - Some(specs.values.maxBy(_.numPartitions)) + Some(finalCandidateSpecs.values.maxBy(_.numPartitions)) } children = children.zip(requiredChildDistributions).zipWithIndex.map { From a0358bb9b32117964e7f79726dd2af7fc721a119 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Mon, 6 Dec 2021 11:49:39 -0800 Subject: [PATCH 23/29] fix and add tests --- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala index 8a716305fbca8..6c4e7835267b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala @@ -347,6 +347,17 @@ class ShuffleSpecSuite extends SparkFunSuite { ) } + test("canCreatePartitioning") { + val distribution = ClusteredDistribution(Seq($"a", $"b")) + assert(HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution).canCreatePartitioning) + assert(SinglePartitionShuffleSpec.canCreatePartitioning) + assert(ShuffleSpecCollection(Seq( + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution), + HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), distribution))) + .canCreatePartitioning) + assert(!RangeShuffleSpec(10, distribution).canCreatePartitioning) + } + test("createPartitioning: HashShuffleSpec") { checkCreatePartitioning( HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), @@ -377,26 +388,28 @@ class ShuffleSpecSuite extends SparkFunSuite { } test("createPartitioning: other specs") { + val distribution = ClusteredDistribution(Seq($"a", $"b")) checkCreatePartitioning(SinglePartitionShuffleSpec, - ClusteredDistribution(Seq($"a", $"b")), + distribution, SinglePartition ) checkCreatePartitioning(SinglePartitionShuffleSpec, - ClusteredDistribution(Seq($"a", $"b")), + distribution, SinglePartition ) - checkCreatePartitioning(RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), - ClusteredDistribution(Seq($"c", $"d")), - HashPartitioning(Seq($"c", $"d"), 10) - ) - checkCreatePartitioning(ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), - RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))))), + HashShuffleSpec(HashPartitioning(Seq($"a"), 10), distribution), + RangeShuffleSpec(10, distribution))), ClusteredDistribution(Seq($"c", $"d")), HashPartitioning(Seq($"c"), 10) ) + + // unsupported cases + + val msg = intercept[Exception](RangeShuffleSpec(10, distribution) + .createPartitioning(distribution.clustering)) + assert(msg.getMessage.contains("Operation unsupported")) } } From 2511d5d60f16321965fb40d9db6cb68e3b2ef4b2 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 8 Dec 2021 10:29:22 -0800 Subject: [PATCH 24/29] fix logic to insert shuffle --- .../exchange/EnsureRequirements.scala | 16 +- .../exchange/EnsureRequirementsSuite.scala | 142 +++++++++--------- 2 files changed, 81 insertions(+), 77 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 577a5a67b1593..4e31ac5e3fcab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -94,18 +94,18 @@ case class EnsureRequirements( // always have exactly 2 children. // Whether we should consider `spark.sql.shuffle.partitions` and ensure enough parallelism - // during the shuffle. To achieve a good trade-off between parallelism and shuffle cost, we - // only consider the minimum parallelism if: - // 1. Some child can't create partitioning, i.e., it needs to be shuffled. - // 2. Some child already needs to be shuffled with `ShuffleExchangeExec` being present. - // In either of the above cases, we'll apply `spark.sql.shuffle.partitions` in case there - // is not enough parallelism. + // during shuffle. To achieve a good trade-off between parallelism and shuffle cost, we only + // consider the minimum parallelism iff ALL children need to be re-shuffled. // - // On the other hand, if we have: + // A child is considered to be re-shuffled iff: + // 1. It can't create partitioning by itself, i.e., `canCreatePartitioning` returns false. + // 2. It already has `ShuffleExchangeExec`. + // + // On the other hand, in scenarios such as: // HashPartitioning(5) <-> HashPartitioning(6) // while `spark.sql.shuffle.partitions` is 10, we'll only re-shuffle the left side and make it // HashPartitioning(6). - val canIgnoreMinPartitions = specs.forall(p => + val canIgnoreMinPartitions = specs.exists(p => p._2.canCreatePartitioning && !children(p._1).isInstanceOf[ShuffleExchangeExec] ) // Choose all the specs that can be used to shuffle other children diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index c13581cbc1545..1e3a74bc70a46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -241,7 +241,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: incompatible & repartitioning with HashShuffleSpec") { + test("incompatible & repartitioning with HashShuffleSpec") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> 5.toString) { var plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 10)) @@ -259,7 +259,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } - // rhs has more partitions so should be chosen + // RHS has more partitions so should be chosen plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 5)) plan2 = DummySparkPlan( @@ -276,7 +276,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } - // if both sides have the same # of partitions, should pick the first one from left + // If both sides have the same # of partitions, should pick the first one from left plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 5)) plan2 = DummySparkPlan( @@ -297,7 +297,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: incompatible & repartitioning with HashShuffleSpec and duplicate keys") { + test("Incompatible & repartitioning with HashShuffleSpec and duplicate keys") { var plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 10)) var plan2 = DummySparkPlan( @@ -331,7 +331,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: successful compatibility check with other specs") { + test("Successful compatibility check with other specs") { var plan1 = DummySparkPlan(outputPartitioning = SinglePartition) var plan2 = DummySparkPlan(outputPartitioning = SinglePartition) var smjExec = SortMergeJoinExec( @@ -367,22 +367,26 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: incompatible & repartitioning with other specs") { + test("Incompatible & repartitioning with other specs") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> 5.toString) { + + // SinglePartition <-> RangePartitioning(10) + // Only RHS should be shuffled and be converted to SinglePartition <-> SinglePartition var plan1 = DummySparkPlan(outputPartitioning = SinglePartition) var plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 10)) var smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == conf.numShufflePartitions) - assert(right.numPartitions == conf.numShufflePartitions) + SortExec(_, _, DummySparkPlan(_, _, SinglePartition, _, _), _), + SortExec(_, _, ShuffleExchangeExec(SinglePartition, _, _), _), _) => case other => fail(other.toString) } + // HashPartitioning(10) <-> RangePartitioning(5) + // Only RHS should be shuffled and be converted to + // HashPartitioning(10) <-> HashPartitioning(10) plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA, exprB), 10)) plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 5)) @@ -398,21 +402,26 @@ class EnsureRequirementsSuite extends SharedSparkSession { case other => fail(other.toString) } + // HashPartitioning(1) <-> RangePartitioning(10) + // Only RHS should be shuffled and be converted to HashPartitioning(1) <-> HashPartitioning(1) plan1 = DummySparkPlan(outputPartitioning = HashPartitioning(Seq(exprA), 1)) plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 1)) + Seq(SortOrder.apply(exprC, Ascending, sameOrderExpressions = Seq.empty)), 10)) smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { case SortMergeJoinExec(_, _, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(left.numPartitions == conf.numShufflePartitions) - assert(right.numPartitions == conf.numShufflePartitions) - assert(right.expressions == Seq(exprC, exprD)) + assert(left.numPartitions == 1) + assert(right.numPartitions == 1) + assert(right.expressions == Seq(exprC)) case other => fail(other.toString) } + // RangePartitioning(1) <-> RangePartitioning(1) + // Both sides should be shuffled and be converted to + // HashPartitioning(5) <-> HashPartitioning(5) plan1 = DummySparkPlan(outputPartitioning = RangePartitioning( Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 1)) plan2 = DummySparkPlan(outputPartitioning = RangePartitioning( @@ -466,24 +475,29 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: EnsureRequirements should respect spark.sql.shuffle.partitions") { + test("EnsureRequirements should respect spark.sql.shuffle.partitions") { val defaultNumPartitions = 10 withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> defaultNumPartitions.toString) { + + // HashPartitioning(5) <-> HashPartitioning(5) + // No shuffle should be inserted var plan1: SparkPlan = DummySparkPlan( - outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) var plan2: SparkPlan = DummySparkPlan( - outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) + outputPartitioning = HashPartitioning(exprC :: Nil, 5)) var smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, - SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), - SortExec(_, _, DummySparkPlan(_, _, _: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) + case SortMergeJoinExec(_, _, _, _, + SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), + SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) => + assert(left.expressions === Seq(exprA)) + assert(right.expressions === Seq(exprC)) case other => fail(other.toString) } + // HashPartitioning(6) <-> HashPartitioning(5) + // Should shuffle RHS and convert to HashPartitioning(6) <-> HashPartitioning(6) plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 6)) plan2 = DummySparkPlan( @@ -491,50 +505,56 @@ class EnsureRequirementsSuite extends SharedSparkSession { smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, + case SortMergeJoinExec(_, _, _, _, SortExec(_, _, DummySparkPlan(_, _, left: HashPartitioning, _, _), _), SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) + assert(left.expressions === Seq(exprA, exprB)) + assert(right.expressions === Seq(exprC, exprD)) assert(left.numPartitions == 6) assert(right.numPartitions == 6) case other => fail(other.toString) } + // RangePartitioning(10) <-> HashPartitioning(5) + // Should shuffle LHS and convert to HashPartitioning(5) <-> HashPartitioning(5) plan1 = DummySparkPlan( outputPartitioning = RangePartitioning( - Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 1)) + Seq(SortOrder.apply(exprA, Ascending, sameOrderExpressions = Seq.empty)), 10)) plan2 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) + outputPartitioning = HashPartitioning(exprD :: Nil, 5)) smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, + case SortMergeJoinExec(_, _, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(left.numPartitions == defaultNumPartitions) - assert(right.numPartitions == defaultNumPartitions) + SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) => + assert(left.expressions === Seq(exprB)) + assert(right.expressions === Seq(exprD)) + assert(left.numPartitions == 5) + assert(right.numPartitions == 5) case other => fail(other.toString) } + // SinglePartition <-> HashPartitioning(5) + // Should shuffle LHS and convert to HashPartitioning(5) <-> HashPartitioning(5) plan1 = DummySparkPlan(outputPartitioning = SinglePartition) plan2 = DummySparkPlan( outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 5)) smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, + case SortMergeJoinExec(_, _, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) + assert(left.expressions === Seq(exprA, exprB)) + assert(right.expressions === Seq(exprC, exprD)) assert(left.numPartitions == 5) assert(right.numPartitions == 5) case other => fail(other.toString) } + // ShuffleExchange(7) <-> HashPartitioning(6) + // Should shuffle LHS and convert to HashPartitioning(6) <-> HashPartitioning(6) plan1 = ShuffleExchangeExec( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 7), child = DummySparkPlan()) @@ -543,51 +563,35 @@ class EnsureRequirementsSuite extends SharedSparkSession { smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, - SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), - SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(left.numPartitions == defaultNumPartitions) - assert(right.numPartitions == defaultNumPartitions) - case other => fail(other.toString) - } - - plan1 = ShuffleExchangeExec( - outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 20), - child = DummySparkPlan()) - plan2 = DummySparkPlan( - outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 11)) - smjExec = SortMergeJoinExec( - exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) - EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, + case SortMergeJoinExec(_, _, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), SortExec(_, _, DummySparkPlan(_, _, right: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(left.numPartitions == 11) - assert(right.numPartitions == 11) + assert(left.expressions === Seq(exprA, exprB)) + assert(right.expressions === Seq(exprC, exprD)) + assert(left.numPartitions == 6) + assert(right.numPartitions == 6) case other => fail(other.toString) } - // if both sides already have shuffle, we should consider `conf.numShufflePartitions` + // ShuffleExchange(7) <-> ShuffleExchange(6) + // Should consider `spark.sql.shuffle.partitions` and shuffle both sides, and + // convert to HashPartitioning(10) <-> HashPartitioning(10) plan1 = ShuffleExchangeExec( - outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5), + outputPartitioning = HashPartitioning(exprA :: Nil, 7), child = DummySparkPlan()) plan2 = ShuffleExchangeExec( - outputPartitioning = HashPartitioning(exprC :: exprD :: Nil, 7), + outputPartitioning = HashPartitioning(exprC :: Nil, 6), child = DummySparkPlan()) smjExec = SortMergeJoinExec( exprA :: exprB :: Nil, exprC :: exprD :: Nil, Inner, None, plan1, plan2) EnsureRequirements.apply(smjExec) match { - case SortMergeJoinExec(leftKeys, rightKeys, _, _, + case SortMergeJoinExec(_, _, _, _, SortExec(_, _, ShuffleExchangeExec(left: HashPartitioning, _, _), _), SortExec(_, _, ShuffleExchangeExec(right: HashPartitioning, _, _), _), _) => - assert(leftKeys === Seq(exprA, exprB)) - assert(rightKeys === Seq(exprC, exprD)) - assert(left.numPartitions == 10) - assert(right.numPartitions == 10) + assert(left.expressions === Seq(exprA, exprB)) + assert(right.expressions === Seq(exprC, exprD)) + assert(left.numPartitions == conf.numShufflePartitions) + assert(right.numPartitions == conf.numShufflePartitions) case other => fail(other.toString) } } From bbf2475fea34275d88b53b69925afe5bb020e6db Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 8 Dec 2021 13:43:35 -0800 Subject: [PATCH 25/29] Fix test failures --- .../test/scala/org/apache/spark/sql/CachedTableSuite.scala | 6 ++---- .../scala/org/apache/spark/sql/execution/PlannerSuite.scala | 4 ++-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 52803e2a9cff8..4de409f56d043 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -604,9 +604,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils uncacheTable("t2") } - // One side of join is not partitioned in the desired way. Since the number of partitions of - // the side that has already partitioned is smaller than the side that is not partitioned, - // we shuffle both side. + // One side of join is not partitioned in the desired way. We'll only shuffle this side. withTempView("t1", "t2") { testData.repartition(6, $"value").createOrReplaceTempView("t1") testData2.repartition(3, $"a").createOrReplaceTempView("t2") @@ -614,7 +612,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils spark.catalog.cacheTable("t2") val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") - verifyNumExchanges(query, 2) + verifyNumExchanges(query, 1) checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 11a453d831339..3bda5625471b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -257,7 +257,7 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { val numExchanges = collect(plan) { case exchange: ShuffleExchangeExec => exchange }.length - assert(numExchanges === 5) + assert(numExchanges === 3) } { @@ -273,7 +273,7 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { val numExchanges = collect(plan) { case exchange: ShuffleExchangeExec => exchange }.length - assert(numExchanges === 5) + assert(numExchanges === 3) } } From 9ac2656820316318022d6ccd702cb148d152c8a3 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Mon, 20 Dec 2021 14:01:38 -0800 Subject: [PATCH 26/29] address comments --- .../plans/physical/partitioning.scala | 29 ++++++++++++------- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 2 +- .../exchange/EnsureRequirements.scala | 22 +++++++------- 3 files changed, 31 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 1ebdc5640800d..7d30ecd97c3ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -356,10 +356,10 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { * of which have their own requirement regarding whether its data can be considered as * co-partitioned from others. This offers APIs for: * - * 1. Comparing with specs from other children of the operator and check if they are compatible. + * - Comparing with specs from other children of the operator and check if they are compatible. * When two specs are compatible, we can say their data are co-partitioned, and Spark will - * potentially able to eliminate shuffle if necessary. - * 1. Creating a partitioning that can be used to re-partition another child, so that to make it + * potentially be able to eliminate shuffle if necessary. + * - Creating a partitioning that can be used to re-partition another child, so that to make it * having a compatible partitioning as this node. */ trait ShuffleSpec { @@ -383,7 +383,7 @@ trait ShuffleSpec { def canCreatePartitioning: Boolean = false /** - * Creates a partitioning that can be used to re-partitioned the other side with the given + * Creates a partitioning that can be used to re-partition the other side with the given * clustering expressions. * * This will only be called when: @@ -397,7 +397,7 @@ trait ShuffleSpec { case object SinglePartitionShuffleSpec extends ShuffleSpec { override def isCompatibleWith(other: ShuffleSpec): Boolean = { - other.numPartitions == numPartitions + other.numPartitions == 1 } override def canCreatePartitioning: Boolean = true @@ -415,6 +415,8 @@ case class RangeShuffleSpec( override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { case SinglePartitionShuffleSpec => numPartitions == 1 case ShuffleSpecCollection(specs) => specs.exists(isCompatibleWith) + // `RangePartitioning` is not compatible with any other partitioning since it can't guarantee + // data are co-partitioned for all the children, as range boundaries are randomly sampled. case _ => false } } @@ -422,7 +424,7 @@ case class RangeShuffleSpec( case class HashShuffleSpec( partitioning: HashPartitioning, distribution: ClusteredDistribution) extends ShuffleSpec { - lazy val hashKeyPositions = + lazy val hashKeyPositions: Seq[mutable.BitSet] = createHashKeyPositions(distribution.clustering, partitioning.expressions) override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { @@ -430,10 +432,12 @@ case class HashShuffleSpec( partitioning.numPartitions == 1 case otherHashSpec @ HashShuffleSpec(otherPartitioning, otherDistribution) => // we need to check: - // 1. both partitioning have the same number of partitions - // 2. both partitioning have the same number of expressions - // 3. each pair of expression from both has overlapping positions in their + // 1. both distributions have the same number of clustering expressions + // 2. both partitioning have the same number of partitions + // 3. both partitioning have the same number of expressions + // 4. each pair of expression from both has overlapping positions in their // corresponding distributions. + distribution.clustering.length == otherDistribution.clustering.length && partitioning.numPartitions == otherPartitioning.numPartitions && partitioning.expressions.length == otherPartitioning.expressions.length && { val otherHashKeyPositions = otherHashSpec.hashKeyPositions @@ -484,8 +488,13 @@ case class ShuffleSpecCollection(specs: Seq[ShuffleSpec]) extends ShuffleSpec { override def createPartitioning(clustering: Seq[Expression]): Partitioning = { // as we only consider # of partitions as the cost now, it doesn't matter which one we choose // since they should all have the same # of partitions. + require(specs.map(_.numPartitions).toSet.size == 1, "expected all specs in the collection " + + "to have the same number of partitions") specs.head.createPartitioning(clustering) } - override def numPartitions: Int = specs.head.numPartitions + override def numPartitions: Int = { + require(specs.nonEmpty, "expected specs to be non-empty") + specs.head.numPartitions + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala index 6c4e7835267b4..d4d73b363e23d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala @@ -35,7 +35,7 @@ class ShuffleSpecSuite extends SparkFunSuite { |$left |== Right ShuffleSpec |$right - |== Does left is compatible with right? == + |== Is left compatible with right? == |Expected $expected but got $actual |""".stripMargin ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 4e31ac5e3fcab..c211df738e92c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -65,8 +65,8 @@ case class EnsureRequirements( ShuffleExchangeExec(distribution.createPartitioning(numPartitions), child, shuffleOrigin) } - // Get the indexes of children which have specified distribution requirements and need to have - // same number of partitions. + // Get the indexes of children which have specified distribution requirements and need to be + // co-partitioned. val childrenIndexes = requiredChildDistributions.zipWithIndex.filter { case (_: ClusteredDistribution, _) => true case _ => false @@ -99,30 +99,30 @@ case class EnsureRequirements( // // A child is considered to be re-shuffled iff: // 1. It can't create partitioning by itself, i.e., `canCreatePartitioning` returns false. - // 2. It already has `ShuffleExchangeExec`. + // 2. It already has `ShuffleExchangeLike`. // // On the other hand, in scenarios such as: // HashPartitioning(5) <-> HashPartitioning(6) // while `spark.sql.shuffle.partitions` is 10, we'll only re-shuffle the left side and make it // HashPartitioning(6). val canIgnoreMinPartitions = specs.exists(p => - p._2.canCreatePartitioning && !children(p._1).isInstanceOf[ShuffleExchangeExec] + p._2.canCreatePartitioning && !children(p._1).isInstanceOf[ShuffleExchangeLike] ) // Choose all the specs that can be used to shuffle other children val candidateSpecs = specs .filter(_._2.canCreatePartitioning) .filter(p => canIgnoreMinPartitions || children(p._1).outputPartitioning.numPartitions >= conf.defaultNumShufflePartitions) - val bestSpec = if (candidateSpecs.isEmpty) { + val bestSpecOpt = if (candidateSpecs.isEmpty) { None } else { - // When choosing specs, we should consider those children with no `Exchange` node + // When choosing specs, we should consider those children with no `ShuffleExchangeLike` node // first. For instance, if we have: // A: (No_Exchange, 100) <---> B: (Exchange, 120) // it's better to pick A and change B to (Exchange, 100) instead of picking B and insert a // new shuffle for A. val candidateSpecsWithoutShuffle = candidateSpecs.filter { case (k, _) => - !children(k).isInstanceOf[ShuffleExchangeExec] + !children(k).isInstanceOf[ShuffleExchangeLike] } val finalCandidateSpecs = if (candidateSpecsWithoutShuffle.nonEmpty) { candidateSpecsWithoutShuffle @@ -137,14 +137,14 @@ case class EnsureRequirements( case ((child, _), idx) if !childrenIndexes.contains(idx) => child case ((child, dist), idx) => - if (bestSpec.isDefined && bestSpec.get.isCompatibleWith(specs(idx))) { + if (bestSpecOpt.isDefined && bestSpecOpt.get.isCompatibleWith(specs(idx))) { child } else { - val newPartitioning = if (bestSpec.isDefined) { + val newPartitioning = bestSpecOpt.map { bestSpec => // Use the best spec to create a new partitioning to re-shuffle this child val clustering = dist.asInstanceOf[ClusteredDistribution].clustering - bestSpec.get.createPartitioning(clustering) - } else { + bestSpec.createPartitioning(clustering) + }.getOrElse { // No best spec available, so we create default partitioning from the required // distribution val numPartitions = dist.requiredNumPartitions From 24fe4880149fb655ed608836bcc7678b9e908e4d Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Tue, 21 Dec 2021 16:54:10 -0800 Subject: [PATCH 27/29] retrigger build From 3ad0d8cd27aaa48213690049d6387fd5eaa7e3b8 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 22 Dec 2021 09:38:58 -0800 Subject: [PATCH 28/29] minor fixes --- .../sql/execution/exchange/EnsureRequirementsSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala index 1e3a74bc70a46..046ff78ce9bd3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala @@ -138,7 +138,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: successful compatibility check with HashShuffleSpec") { + test("Successful compatibility check with HashShuffleSpec") { val plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: Nil, 5)) val plan2 = DummySparkPlan( @@ -179,7 +179,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { } } - test("SPARK-35703: successful compatibility check with HashShuffleSpec and duplicate keys") { + test("Successful compatibility check with HashShuffleSpec and duplicate keys") { var plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 5)) var plan2 = DummySparkPlan( @@ -600,7 +600,7 @@ class EnsureRequirementsSuite extends SharedSparkSession { test("Respect spark.sql.shuffle.partitions with AQE") { withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> 8.toString, SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> 10.toString) { - Seq(false).foreach { enable => + Seq(true, false).foreach { enable => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> s"$enable") { val plan1 = DummySparkPlan( outputPartitioning = HashPartitioning(exprA :: exprB :: Nil, 9)) From 112b110a5099c6a2e16979f695f06b47671a2e0a Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 23 Dec 2021 09:41:48 -0800 Subject: [PATCH 29/29] address comments --- .../exchange/EnsureRequirements.scala | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index c211df738e92c..de1806ab87b4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -83,12 +83,13 @@ case class EnsureRequirements( requiredDist.asInstanceOf[ClusteredDistribution]) }).toMap - // Find out the shuffle spec that gives better parallelism. + // Find out the shuffle spec that gives better parallelism. Currently this is done by + // picking the spec with the largest number of partitions. // // NOTE: this is not optimal for the case when there are more than 2 children. Consider: // (10, 10, 11) - // it's better to pick 10 in this case since we only need to shuffle one side - we'd need to - // shuffle two sides if we pick 11. + // where the number represent the number of partitions for each child, it's better to pick 10 + // here since we only need to shuffle one side - we'd need to shuffle two sides if we pick 11. // // However this should be sufficient for now since in Spark nodes with multiple children // always have exactly 2 children. @@ -97,21 +98,24 @@ case class EnsureRequirements( // during shuffle. To achieve a good trade-off between parallelism and shuffle cost, we only // consider the minimum parallelism iff ALL children need to be re-shuffled. // - // A child is considered to be re-shuffled iff: - // 1. It can't create partitioning by itself, i.e., `canCreatePartitioning` returns false. - // 2. It already has `ShuffleExchangeLike`. + // A child needs to be re-shuffled iff either one of below is true: + // 1. It can't create partitioning by itself, i.e., `canCreatePartitioning` returns false + // (as for the case of `RangePartitioning`), therefore it needs to be re-shuffled + // according to other shuffle spec. + // 2. It already has `ShuffleExchangeLike`, so we can re-use existing shuffle without + // introducing extra shuffle. // // On the other hand, in scenarios such as: // HashPartitioning(5) <-> HashPartitioning(6) // while `spark.sql.shuffle.partitions` is 10, we'll only re-shuffle the left side and make it // HashPartitioning(6). - val canIgnoreMinPartitions = specs.exists(p => - p._2.canCreatePartitioning && !children(p._1).isInstanceOf[ShuffleExchangeLike] + val shouldConsiderMinParallelism = specs.forall(p => + !p._2.canCreatePartitioning || children(p._1).isInstanceOf[ShuffleExchangeLike] ) // Choose all the specs that can be used to shuffle other children val candidateSpecs = specs .filter(_._2.canCreatePartitioning) - .filter(p => canIgnoreMinPartitions || + .filter(p => !shouldConsiderMinParallelism || children(p._1).outputPartitioning.numPartitions >= conf.defaultNumShufflePartitions) val bestSpecOpt = if (candidateSpecs.isEmpty) { None