From b272551d76d4bba6b30501b5660420368a874145 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 7 Jul 2024 12:52:11 -0700 Subject: [PATCH 1/9] fix: Remove original plan parameter from CometNativeExec --- .../comet/CometSparkSessionExtensions.scala | 178 ++++++++++++------ .../comet/CometBroadcastExchangeExec.scala | 11 +- .../spark/sql/comet/CometCoalesceExec.scala | 3 +- .../sql/comet/CometCollectLimitExec.scala | 10 +- .../CometTakeOrderedAndProjectExec.scala | 4 +- .../spark/sql/comet/CometWindowExec.scala | 4 +- .../shuffle/CometShuffleExchangeExec.scala | 1 - .../apache/spark/sql/comet/operators.scala | 90 ++++----- .../shims/ShimCometShuffleExchangeExec.scala | 1 - .../shims/ShimCometShuffleExchangeExec.scala | 1 - .../approved-plans-v1_4/q1/simplified.txt | 22 +-- .../approved-plans-v1_4/q10/simplified.txt | 22 +-- .../approved-plans-v1_4/q11/simplified.txt | 60 +++--- .../approved-plans-v1_4/q12/simplified.txt | 18 +- .../approved-plans-v1_4/q13/simplified.txt | 30 +-- .../approved-plans-v1_4/q14a/simplified.txt | 56 +++--- .../approved-plans-v1_4/q14b/simplified.txt | 58 +++--- .../approved-plans-v1_4/q15/simplified.txt | 22 +-- .../approved-plans-v1_4/q16/simplified.txt | 8 +- .../approved-plans-v1_4/q17/simplified.txt | 46 ++--- .../approved-plans-v1_4/q18/simplified.txt | 40 ++-- .../approved-plans-v1_4/q19/simplified.txt | 30 +-- .../approved-plans-v1_4/q2/simplified.txt | 14 +- .../approved-plans-v1_4/q20/simplified.txt | 18 +- .../approved-plans-v1_4/q21/simplified.txt | 24 +-- .../approved-plans-v1_4/q22/simplified.txt | 22 +-- .../approved-plans-v1_4/q23a/simplified.txt | 42 ++--- .../approved-plans-v1_4/q23b/simplified.txt | 48 ++--- .../approved-plans-v1_4/q24a/simplified.txt | 14 +- .../approved-plans-v1_4/q24b/simplified.txt | 14 +- .../approved-plans-v1_4/q25/simplified.txt | 46 ++--- .../approved-plans-v1_4/q26/simplified.txt | 30 +-- .../approved-plans-v1_4/q27/simplified.txt | 30 +-- .../approved-plans-v1_4/q28/simplified.txt | 24 +-- .../approved-plans-v1_4/q29/simplified.txt | 52 ++--- .../approved-plans-v1_4/q3/simplified.txt | 16 +- .../approved-plans-v1_4/q30/simplified.txt | 28 +-- .../approved-plans-v1_4/q31/simplified.txt | 68 +++---- .../approved-plans-v1_4/q32/simplified.txt | 20 +- .../approved-plans-v1_4/q33/simplified.txt | 50 ++--- .../approved-plans-v1_4/q34/simplified.txt | 26 +-- .../approved-plans-v1_4/q35/simplified.txt | 22 +-- .../approved-plans-v1_4/q36/simplified.txt | 24 +-- .../approved-plans-v1_4/q37/simplified.txt | 22 +-- .../approved-plans-v1_4/q38/simplified.txt | 28 +-- .../approved-plans-v1_4/q39a/simplified.txt | 38 ++-- .../approved-plans-v1_4/q39b/simplified.txt | 38 ++-- .../approved-plans-v1_4/q4/simplified.txt | 84 ++++----- .../approved-plans-v1_4/q40/simplified.txt | 10 +- .../approved-plans-v1_4/q41/simplified.txt | 6 +- .../approved-plans-v1_4/q42/simplified.txt | 16 +- .../approved-plans-v1_4/q43/simplified.txt | 16 +- .../approved-plans-v1_4/q44/simplified.txt | 10 +- .../approved-plans-v1_4/q45/simplified.txt | 28 +-- .../approved-plans-v1_4/q46/simplified.txt | 30 +-- .../approved-plans-v1_4/q47/simplified.txt | 22 +-- .../approved-plans-v1_4/q48/simplified.txt | 26 +-- .../approved-plans-v1_4/q49/simplified.txt | 42 ++--- .../approved-plans-v1_4/q5/simplified.txt | 58 +++--- .../approved-plans-v1_4/q50/simplified.txt | 28 +-- .../approved-plans-v1_4/q51/simplified.txt | 18 +- .../approved-plans-v1_4/q52/simplified.txt | 16 +- .../approved-plans-v1_4/q53/simplified.txt | 22 +-- .../approved-plans-v1_4/q54/simplified.txt | 32 ++-- .../approved-plans-v1_4/q55/simplified.txt | 16 +- .../approved-plans-v1_4/q56/simplified.txt | 50 ++--- .../approved-plans-v1_4/q57/simplified.txt | 22 +-- .../approved-plans-v1_4/q58/simplified.txt | 42 ++--- .../approved-plans-v1_4/q59/simplified.txt | 12 +- .../approved-plans-v1_4/q6/simplified.txt | 30 +-- .../approved-plans-v1_4/q60/simplified.txt | 50 ++--- .../approved-plans-v1_4/q61/simplified.txt | 54 +++--- .../approved-plans-v1_4/q62/simplified.txt | 28 +-- .../approved-plans-v1_4/q63/simplified.txt | 22 +-- .../approved-plans-v1_4/q64/simplified.txt | 32 ++-- .../approved-plans-v1_4/q65/simplified.txt | 22 +-- .../approved-plans-v1_4/q66/simplified.txt | 42 ++--- .../approved-plans-v1_4/q67/simplified.txt | 24 +-- .../approved-plans-v1_4/q68/simplified.txt | 30 +-- .../approved-plans-v1_4/q69/simplified.txt | 22 +-- .../approved-plans-v1_4/q7/simplified.txt | 30 +-- .../approved-plans-v1_4/q70/simplified.txt | 24 +-- .../approved-plans-v1_4/q71/simplified.txt | 34 ++-- .../approved-plans-v1_4/q72/simplified.txt | 50 ++--- .../approved-plans-v1_4/q73/simplified.txt | 26 +-- .../approved-plans-v1_4/q74/simplified.txt | 60 +++--- .../approved-plans-v1_4/q75/simplified.txt | 58 +++--- .../approved-plans-v1_4/q76/simplified.txt | 30 +-- .../approved-plans-v1_4/q77/simplified.txt | 50 ++--- .../approved-plans-v1_4/q78/simplified.txt | 14 +- .../approved-plans-v1_4/q79/simplified.txt | 26 +-- .../approved-plans-v1_4/q8/simplified.txt | 26 +-- .../approved-plans-v1_4/q80/simplified.txt | 24 +-- .../approved-plans-v1_4/q81/simplified.txt | 28 +-- .../approved-plans-v1_4/q82/simplified.txt | 22 +-- .../approved-plans-v1_4/q83/simplified.txt | 44 ++--- .../approved-plans-v1_4/q84/simplified.txt | 28 +-- .../approved-plans-v1_4/q85/simplified.txt | 42 ++--- .../approved-plans-v1_4/q86/simplified.txt | 18 +- .../approved-plans-v1_4/q87/simplified.txt | 28 +-- .../approved-plans-v1_4/q88/simplified.txt | 104 +++++----- .../approved-plans-v1_4/q89/simplified.txt | 24 +-- .../approved-plans-v1_4/q9/simplified.txt | 20 +- .../approved-plans-v1_4/q90/simplified.txt | 32 ++-- .../approved-plans-v1_4/q91/simplified.txt | 42 ++--- .../approved-plans-v1_4/q92/simplified.txt | 20 +- .../approved-plans-v1_4/q93/simplified.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 8 +- .../approved-plans-v1_4/q95/simplified.txt | 12 +- .../approved-plans-v1_4/q96/simplified.txt | 20 +- .../approved-plans-v1_4/q97/simplified.txt | 10 +- .../approved-plans-v1_4/q98/simplified.txt | 18 +- .../approved-plans-v1_4/q99/simplified.txt | 28 +-- .../approved-plans-v2_7/q10a/simplified.txt | 38 ++-- .../approved-plans-v2_7/q11/simplified.txt | 60 +++--- .../approved-plans-v2_7/q12/simplified.txt | 18 +- .../approved-plans-v2_7/q14/simplified.txt | 58 +++--- .../approved-plans-v2_7/q14a/simplified.txt | 62 +++--- .../approved-plans-v2_7/q18a/simplified.txt | 118 ++++++------ .../approved-plans-v2_7/q20/simplified.txt | 18 +- .../approved-plans-v2_7/q22/simplified.txt | 16 +- .../approved-plans-v2_7/q22a/simplified.txt | 22 +-- .../approved-plans-v2_7/q24/simplified.txt | 14 +- .../approved-plans-v2_7/q27a/simplified.txt | 60 +++--- .../approved-plans-v2_7/q34/simplified.txt | 26 +-- .../approved-plans-v2_7/q35/simplified.txt | 22 +-- .../approved-plans-v2_7/q35a/simplified.txt | 38 ++-- .../approved-plans-v2_7/q36a/simplified.txt | 24 +-- .../approved-plans-v2_7/q47/simplified.txt | 22 +-- .../approved-plans-v2_7/q49/simplified.txt | 42 ++--- .../approved-plans-v2_7/q51a/simplified.txt | 18 +- .../approved-plans-v2_7/q57/simplified.txt | 22 +-- .../approved-plans-v2_7/q5a/simplified.txt | 58 +++--- .../approved-plans-v2_7/q6/simplified.txt | 30 +-- .../approved-plans-v2_7/q64/simplified.txt | 32 ++-- .../approved-plans-v2_7/q67a/simplified.txt | 24 +-- .../approved-plans-v2_7/q70a/simplified.txt | 24 +-- .../approved-plans-v2_7/q72/simplified.txt | 50 ++--- .../approved-plans-v2_7/q74/simplified.txt | 60 +++--- .../approved-plans-v2_7/q75/simplified.txt | 58 +++--- .../approved-plans-v2_7/q77a/simplified.txt | 50 ++--- .../approved-plans-v2_7/q78/simplified.txt | 14 +- .../approved-plans-v2_7/q80a/simplified.txt | 24 +-- .../approved-plans-v2_7/q86a/simplified.txt | 18 +- .../approved-plans-v2_7/q98/simplified.txt | 18 +- 145 files changed, 2312 insertions(+), 2285 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index fb1f62542..e0a23e41e 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -328,13 +328,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometProjectExec( + val newPlan = CometProjectExec( nativeOp, - op, - op.projectList, op.output, + op.projectList, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -343,7 +343,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometFilterExec(nativeOp, op, op.condition, op.child, SerializedPlan(None)) + val newPlan = + CometFilterExec(nativeOp, op.output, op.condition, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -352,7 +354,15 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometSortExec(nativeOp, op, op.sortOrder, op.child, SerializedPlan(None)) + val newPlan = + CometSortExec( + nativeOp, + op.output, + op.outputOrdering, + op.sortOrder, + op.child, + SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -361,7 +371,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometLocalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) + val newPlan = + CometLocalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -370,7 +382,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometGlobalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) + val newPlan = + CometGlobalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -382,8 +396,9 @@ class CometSparkSessionExtensions QueryPlanSerde.operator2Proto(op) match { case Some(nativeOp) => val offset = getOffset(op) - val cometOp = - CometCollectLimitExec(op, op.limit, offset, op.child) + val newPlan = + CometCollectLimitExec(op.limit, offset, op.child) + val cometOp = setLogicalLink(newPlan, op) CometSinkPlaceHolder(nativeOp, op, cometOp) case None => op @@ -393,12 +408,28 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometExpandExec(nativeOp, op, op.projections, op.child, SerializedPlan(None)) + val newPlan = + CometExpandExec( + nativeOp, + op.output, + op.projections, + op.child, + SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } - case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) => + case op @ HashAggregateExec( + _, + _, + _, + groupingExprs, + aggExprs, + _, + _, + resultExpressions, + child) => val modes = aggExprs.map(_.mode).distinct if (!modes.isEmpty && modes.size != 1) { @@ -422,15 +453,17 @@ class CometSparkSessionExtensions // modes is empty too. If aggExprs is not empty, we need to verify all the // aggregates have the same mode. assert(modes.length == 1 || modes.length == 0) - CometHashAggregateExec( + val newPlan = CometHashAggregateExec( nativeOp, - op, + op.output, groupingExprs, aggExprs, + resultExpressions, child.output, if (modes.nonEmpty) Some(modes.head) else None, child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -443,9 +476,10 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometHashJoinExec( + val newPlan = CometHashJoinExec( nativeOp, - op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -454,6 +488,7 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -475,9 +510,10 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometBroadcastHashJoinExec( + val newPlan = CometBroadcastHashJoinExec( nativeOp, - op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -486,6 +522,7 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -496,9 +533,10 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometSortMergeJoinExec( + val newPlan = CometSortMergeJoinExec( nativeOp, - op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -506,6 +544,7 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -535,7 +574,8 @@ class CometSparkSessionExtensions && isCometNative(child) => QueryPlanSerde.operator2Proto(c) match { case Some(nativeOp) => - val cometOp = CometCoalesceExec(c, numPartitions, child) + val newPlan = CometCoalesceExec(c.output, numPartitions, child) + val cometOp = setLogicalLink(newPlan, c) CometSinkPlaceHolder(nativeOp, c, cometOp) case None => c @@ -558,8 +598,14 @@ class CometSparkSessionExtensions CometTakeOrderedAndProjectExec.isSupported(s) => QueryPlanSerde.operator2Proto(s) match { case Some(nativeOp) => - val cometOp = - CometTakeOrderedAndProjectExec(s, s.limit, s.sortOrder, s.projectList, s.child) + val newPlan = + CometTakeOrderedAndProjectExec( + s.output, + s.limit, + s.sortOrder, + s.projectList, + s.child) + val cometOp = setLogicalLink(newPlan, s) CometSinkPlaceHolder(nativeOp, s, cometOp) case None => s @@ -579,8 +625,14 @@ class CometSparkSessionExtensions val newOp = transform1(w) newOp match { case Some(nativeOp) => - val cometOp = - CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) + val newPlan = + CometWindowExec( + w.output, + w.windowExpression, + w.partitionSpec, + w.orderSpec, + w.child) + val cometOp = setLogicalLink(newPlan, w) CometSinkPlaceHolder(nativeOp, w, cometOp) case None => w @@ -591,7 +643,8 @@ class CometSparkSessionExtensions u.children.forall(isCometNative) => QueryPlanSerde.operator2Proto(u) match { case Some(nativeOp) => - val cometOp = CometUnionExec(u, u.children) + val newPlan = CometUnionExec(u.output, u.children) + val cometOp = setLogicalLink(newPlan, u) CometSinkPlaceHolder(nativeOp, u, cometOp) case None => u @@ -631,7 +684,8 @@ class CometSparkSessionExtensions isSpark34Plus => // Spark 3.4+ only QueryPlanSerde.operator2Proto(b) match { case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.child) + val newPlan = CometBroadcastExchangeExec(b.output, b.child) + val cometOp = setLogicalLink(newPlan, b) CometSinkPlaceHolder(nativeOp, b, cometOp) case None => b } @@ -822,40 +876,6 @@ class CometSparkSessionExtensions case CometScanWrapper(_, s) => s } - // Set up logical links - newPlan = newPlan.transform { - case op: CometExec => - if (op.originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - op.originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - case op: CometShuffleExchangeExec => - // Original Spark shuffle exchange operator might have empty logical link. - // But the `setLogicalLink` call above on downstream operator of - // `CometShuffleExchangeExec` will set its logical link to the downstream - // operators which cause AQE behavior to be incorrect. So we need to unset - // the logical link here. - if (op.originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - op.originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - - case op: CometBroadcastExchangeExec => - if (op.originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - op.originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - } - // Convert native execution block by linking consecutive native operators. var firstNativeOp = true newPlan.transformDown { @@ -887,6 +907,44 @@ class CometSparkSessionExtensions }.flatten } + /** + * Set up logical links for transformed Comet operators. + */ + def setLogicalLink(newPlan: SparkPlan, originalPlan: SparkPlan): SparkPlan = { + newPlan match { + case op: CometExec => + if (originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + case op: CometShuffleExchangeExec => + // Original Spark shuffle exchange operator might have empty logical link. + // But the `setLogicalLink` call above on downstream operator of + // `CometShuffleExchangeExec` will set its logical link to the downstream + // operators which cause AQE behavior to be incorrect. So we need to unset + // the logical link here. + if (originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + + case op: CometBroadcastExchangeExec => + if (originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + } + } + /** * Returns true if a given spark plan is Comet shuffle operator. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index 38247b2c4..426806916 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -31,6 +31,7 @@ import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} @@ -60,7 +61,9 @@ import org.apache.comet.CometRuntimeException * Note that this only supports Spark 3.4 and later, because the serialization class * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later. */ -case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) +case class CometBroadcastExchangeExec( + override val output: Seq[Attribute], + override val child: SparkPlan) extends BroadcastExchangeLike with ShimCometBroadcastExchangeExec { import CometBroadcastExchangeExec._ @@ -74,10 +77,6 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) - override def doCanonicalize(): SparkPlan = { - CometBroadcastExchangeExec(originalPlan.canonicalized, child.canonicalized) - } - override def runtimeStatistics: Statistics = { val dataSize = metrics("dataSize").value val rowCount = metrics("numOutputRows").value @@ -237,7 +236,7 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) override def equals(obj: Any): Boolean = { obj match { case other: CometBroadcastExchangeExec => - this.originalPlan == other.originalPlan && + this.output == other.output && this.child == other.child case _ => false diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala index cc635d739..1f656b117 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -31,7 +32,7 @@ import com.google.common.base.Objects * more efficient when including it in a Comet query plan. */ case class CometCoalesceExec( - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], numPartitions: Int, child: SparkPlan) extends CometExec diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index d2c9158ee..0090c537c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -22,6 +22,8 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer} @@ -38,13 +40,11 @@ import com.google.common.base.Objects * * TODO: support offset semantics */ -case class CometCollectLimitExec( - override val originalPlan: SparkPlan, - limit: Int, - offset: Int, - child: SparkPlan) +case class CometCollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends CometExec with UnaryExecNode { + override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 6e9bfe424..2f5121bed 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -41,15 +41,13 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec * contains two native executions separated by a Comet shuffle exchange. */ case class CometTakeOrderedAndProjectExec( - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], limit: Int, sortOrder: Seq[SortOrder], projectList: Seq[NamedExpression], child: SparkPlan) extends CometExec with UnaryExecNode { - override def output: Seq[Attribute] = projectList.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 9685e75e1..221b725d5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -42,7 +42,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi * executions separated by a Comet shuffle exchange. */ case class CometWindowExec( - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], @@ -52,8 +52,6 @@ case class CometWindowExec( override def nodeName: String = "CometWindowExec" - override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index cec64d7a8..5d4848216 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -64,7 +64,6 @@ import org.apache.comet.shims.ShimCometShuffleExchangeExec case class CometShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, - originalPlan: ShuffleExchangeLike, shuffleOrigin: ShuffleOrigin = ENSURE_REQUIREMENTS, shuffleType: ShuffleType = CometNativeShuffle, advisoryPartitionSize: Option[Long] = None) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 8b1ba168f..732213158 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf @@ -57,28 +56,15 @@ import org.apache.comet.shims.ShimCometBroadcastHashJoinExec */ abstract class CometExec extends CometPlan { - /** The original Spark operator from which this Comet operator is converted from */ - def originalPlan: SparkPlan - /** Comet always support columnar execution */ override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = originalPlan.output - override def doExecute(): RDD[InternalRow] = ColumnarToRowExec(this).doExecute() override def executeCollect(): Array[InternalRow] = ColumnarToRowExec(this).executeCollect() - override def outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering - - // `CometExec` reuses the outputPartitioning of the original SparkPlan. - // Note that if the outputPartitioning of the original SparkPlan depends on its children, - // we should override this method in the specific CometExec, because Spark AQE may change the - // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. - override def outputPartitioning: Partitioning = originalPlan.outputPartitioning - /** * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. */ @@ -154,7 +140,7 @@ abstract class CometNativeExec extends CometExec { /** The Comet native operator */ def nativeOp: Operator - override protected def doPrepare(): Unit = prepareSubqueries(originalPlan) + override protected def doPrepare(): Unit = prepareSubqueries(this) override lazy val metrics: Map[String, SQLMetric] = CometMetricNode.baselineMetrics(sparkContext) @@ -217,12 +203,12 @@ abstract class CometNativeExec extends CometExec { val it = new CometExecIterator(CometExec.newIterId, inputs, serializedPlanCopy, nativeMetrics) - setSubqueries(it.id, originalPlan) + setSubqueries(it.id, this) Option(TaskContext.get()).foreach { context => context.addTaskCompletionListener[Unit] { _ => it.close() - cleanSubqueries(it.id, originalPlan) + cleanSubqueries(it.id, this) } } @@ -363,10 +349,7 @@ abstract class CometNativeExec extends CometExec { } override protected def doCanonicalize(): SparkPlan = { - val canonicalizedPlan = super - .doCanonicalize() - .asInstanceOf[CometNativeExec] - .canonicalizePlans() + val canonicalizedPlan = super.doCanonicalize().asInstanceOf[CometNativeExec] if (serializedPlanOpt.isDefined) { // If the plan is a boundary node, we should remove the serialized plan. @@ -375,24 +358,6 @@ abstract class CometNativeExec extends CometExec { canonicalizedPlan } } - - /** - * Canonicalizes the plans of Product parameters in Comet native operators. - */ - protected def canonicalizePlans(): CometNativeExec = { - def transform(arg: Any): AnyRef = arg match { - case sparkPlan: SparkPlan if !sparkPlan.isInstanceOf[CometNativeExec] => - // Different to Spark, Comet native query node might have a Spark plan as Product element. - // We need to canonicalize the Spark plan. But it cannot be another Comet native query node, - // otherwise it will cause recursive canonicalization. - sparkPlan.canonicalized - case other: AnyRef => other - case null => null - } - - val newArgs = mapProductIterator(transform) - makeCopy(newArgs).asInstanceOf[CometNativeExec] - } } abstract class CometUnaryExec extends CometNativeExec with UnaryExecNode @@ -411,9 +376,8 @@ case class SerializedPlan(plan: Option[Array[Byte]]) { case class CometProjectExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, - projectList: Seq[NamedExpression], override val output: Seq[Attribute], + projectList: Seq[NamedExpression], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec @@ -442,19 +406,20 @@ case class CometProjectExec( case class CometFilterExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], condition: Expression, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, condition, child) + Iterator(output, condition, child) override def equals(obj: Any): Boolean = { obj match { @@ -479,7 +444,8 @@ case class CometFilterExec( case class CometSortExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], sortOrder: Seq[SortOrder], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -491,7 +457,7 @@ case class CometSortExec( this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, sortOrder, child) + Iterator(output, sortOrder, child) override def equals(obj: Any): Boolean = { obj match { @@ -514,13 +480,14 @@ case class CometSortExec( case class CometLocalLimitExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -544,13 +511,14 @@ case class CometLocalLimitExec( case class CometGlobalLimitExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -572,11 +540,13 @@ case class CometGlobalLimitExec( case class CometExpandExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], projections: Seq[Seq[Expression]], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def outputPartitioning: Partitioning = UnknownPartitioning(0) + override def producedAttributes: AttributeSet = outputSet override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -600,7 +570,7 @@ case class CometExpandExec( override lazy val metrics: Map[String, SQLMetric] = Map.empty } -case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan]) +case class CometUnionExec(override val output: Seq[Attribute], children: Seq[SparkPlan]) extends CometExec { override def doExecuteColumnar(): RDD[ColumnarBatch] = { sparkContext.union(children.map(_.executeColumnar())) @@ -633,9 +603,10 @@ case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[Sp case class CometHashAggregateExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression], input: Seq[Attribute], mode: Option[AggregateMode], child: SparkPlan, @@ -674,13 +645,13 @@ case class CometHashAggregateExec( override def hashCode(): Int = Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child) - override protected def outputExpressions: Seq[NamedExpression] = - originalPlan.asInstanceOf[HashAggregateExec].resultExpressions + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions } case class CometHashJoinExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -732,7 +703,8 @@ case class CometHashJoinExec( case class CometBroadcastHashJoinExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -866,7 +838,8 @@ case class CometBroadcastHashJoinExec( case class CometSortMergeJoinExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -921,9 +894,10 @@ case class CometSortMergeJoinExec( "join_time" -> SQLMetrics.createNanoTimingMetric(sparkContext, "Total time for joining")) } -case class CometScanWrapper(override val nativeOp: Operator, override val originalPlan: SparkPlan) +case class CometScanWrapper(override val nativeOp: Operator, originalPlan: SparkPlan) extends CometNativeExec with LeafExecNode { + override def output: Seq[Attribute] = originalPlan.output override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override def stringArgs: Iterator[Any] = Iterator(originalPlan.output, originalPlan) } @@ -937,9 +911,11 @@ case class CometScanWrapper(override val nativeOp: Operator, override val origin */ case class CometSinkPlaceHolder( override val nativeOp: Operator, // Must be a Scan - override val originalPlan: SparkPlan, + originalPlan: SparkPlan, child: SparkPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = originalPlan.output + override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 965b6851e..9fadcc680 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -35,7 +35,6 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, - s, s.shuffleOrigin, shuffleType, advisoryPartitionSize) diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 559e327b4..2be852ae5 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -31,7 +31,6 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, - s, s.shuffleOrigin, shuffleType, s.advisoryPartitionSize) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 000160518..238f9b702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [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,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 50472502e..60cad82dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 701a90912..f713176f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -27,9 +27,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -37,9 +37,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -57,7 +57,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index c4ea8fe24..537cba446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index b529cb5ff..c37569802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,15 +30,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index f4a03a2d2..f95e8d040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -57,15 +57,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index f80554fb5..e576a079b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index ecae29c78..2313467ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index dd66c9582..1428c7ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index a3a99e312..dfa9ae66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,17 +30,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 9e72edbb4..feca96b18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -64,16 +64,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] From 722dc07c75407edbf671d47f23769d4e543dbab5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 7 Jul 2024 17:06:14 -0700 Subject: [PATCH 2/9] Revert "fix: Remove original plan parameter from CometNativeExec" This reverts commit b272551d76d4bba6b30501b5660420368a874145. --- .../comet/CometSparkSessionExtensions.scala | 178 ++++++------------ .../comet/CometBroadcastExchangeExec.scala | 11 +- .../spark/sql/comet/CometCoalesceExec.scala | 3 +- .../sql/comet/CometCollectLimitExec.scala | 10 +- .../CometTakeOrderedAndProjectExec.scala | 4 +- .../spark/sql/comet/CometWindowExec.scala | 4 +- .../shuffle/CometShuffleExchangeExec.scala | 1 + .../apache/spark/sql/comet/operators.scala | 90 +++++---- .../shims/ShimCometShuffleExchangeExec.scala | 1 + .../shims/ShimCometShuffleExchangeExec.scala | 1 + .../approved-plans-v1_4/q1/simplified.txt | 22 +-- .../approved-plans-v1_4/q10/simplified.txt | 22 +-- .../approved-plans-v1_4/q11/simplified.txt | 60 +++--- .../approved-plans-v1_4/q12/simplified.txt | 18 +- .../approved-plans-v1_4/q13/simplified.txt | 30 +-- .../approved-plans-v1_4/q14a/simplified.txt | 56 +++--- .../approved-plans-v1_4/q14b/simplified.txt | 58 +++--- .../approved-plans-v1_4/q15/simplified.txt | 22 +-- .../approved-plans-v1_4/q16/simplified.txt | 8 +- .../approved-plans-v1_4/q17/simplified.txt | 46 ++--- .../approved-plans-v1_4/q18/simplified.txt | 40 ++-- .../approved-plans-v1_4/q19/simplified.txt | 30 +-- .../approved-plans-v1_4/q2/simplified.txt | 14 +- .../approved-plans-v1_4/q20/simplified.txt | 18 +- .../approved-plans-v1_4/q21/simplified.txt | 24 +-- .../approved-plans-v1_4/q22/simplified.txt | 22 +-- .../approved-plans-v1_4/q23a/simplified.txt | 42 ++--- .../approved-plans-v1_4/q23b/simplified.txt | 48 ++--- .../approved-plans-v1_4/q24a/simplified.txt | 14 +- .../approved-plans-v1_4/q24b/simplified.txt | 14 +- .../approved-plans-v1_4/q25/simplified.txt | 46 ++--- .../approved-plans-v1_4/q26/simplified.txt | 30 +-- .../approved-plans-v1_4/q27/simplified.txt | 30 +-- .../approved-plans-v1_4/q28/simplified.txt | 24 +-- .../approved-plans-v1_4/q29/simplified.txt | 52 ++--- .../approved-plans-v1_4/q3/simplified.txt | 16 +- .../approved-plans-v1_4/q30/simplified.txt | 28 +-- .../approved-plans-v1_4/q31/simplified.txt | 68 +++---- .../approved-plans-v1_4/q32/simplified.txt | 20 +- .../approved-plans-v1_4/q33/simplified.txt | 50 ++--- .../approved-plans-v1_4/q34/simplified.txt | 26 +-- .../approved-plans-v1_4/q35/simplified.txt | 22 +-- .../approved-plans-v1_4/q36/simplified.txt | 24 +-- .../approved-plans-v1_4/q37/simplified.txt | 22 +-- .../approved-plans-v1_4/q38/simplified.txt | 28 +-- .../approved-plans-v1_4/q39a/simplified.txt | 38 ++-- .../approved-plans-v1_4/q39b/simplified.txt | 38 ++-- .../approved-plans-v1_4/q4/simplified.txt | 84 ++++----- .../approved-plans-v1_4/q40/simplified.txt | 10 +- .../approved-plans-v1_4/q41/simplified.txt | 6 +- .../approved-plans-v1_4/q42/simplified.txt | 16 +- .../approved-plans-v1_4/q43/simplified.txt | 16 +- .../approved-plans-v1_4/q44/simplified.txt | 10 +- .../approved-plans-v1_4/q45/simplified.txt | 28 +-- .../approved-plans-v1_4/q46/simplified.txt | 30 +-- .../approved-plans-v1_4/q47/simplified.txt | 22 +-- .../approved-plans-v1_4/q48/simplified.txt | 26 +-- .../approved-plans-v1_4/q49/simplified.txt | 42 ++--- .../approved-plans-v1_4/q5/simplified.txt | 58 +++--- .../approved-plans-v1_4/q50/simplified.txt | 28 +-- .../approved-plans-v1_4/q51/simplified.txt | 18 +- .../approved-plans-v1_4/q52/simplified.txt | 16 +- .../approved-plans-v1_4/q53/simplified.txt | 22 +-- .../approved-plans-v1_4/q54/simplified.txt | 32 ++-- .../approved-plans-v1_4/q55/simplified.txt | 16 +- .../approved-plans-v1_4/q56/simplified.txt | 50 ++--- .../approved-plans-v1_4/q57/simplified.txt | 22 +-- .../approved-plans-v1_4/q58/simplified.txt | 42 ++--- .../approved-plans-v1_4/q59/simplified.txt | 12 +- .../approved-plans-v1_4/q6/simplified.txt | 30 +-- .../approved-plans-v1_4/q60/simplified.txt | 50 ++--- .../approved-plans-v1_4/q61/simplified.txt | 54 +++--- .../approved-plans-v1_4/q62/simplified.txt | 28 +-- .../approved-plans-v1_4/q63/simplified.txt | 22 +-- .../approved-plans-v1_4/q64/simplified.txt | 32 ++-- .../approved-plans-v1_4/q65/simplified.txt | 22 +-- .../approved-plans-v1_4/q66/simplified.txt | 42 ++--- .../approved-plans-v1_4/q67/simplified.txt | 24 +-- .../approved-plans-v1_4/q68/simplified.txt | 30 +-- .../approved-plans-v1_4/q69/simplified.txt | 22 +-- .../approved-plans-v1_4/q7/simplified.txt | 30 +-- .../approved-plans-v1_4/q70/simplified.txt | 24 +-- .../approved-plans-v1_4/q71/simplified.txt | 34 ++-- .../approved-plans-v1_4/q72/simplified.txt | 50 ++--- .../approved-plans-v1_4/q73/simplified.txt | 26 +-- .../approved-plans-v1_4/q74/simplified.txt | 60 +++--- .../approved-plans-v1_4/q75/simplified.txt | 58 +++--- .../approved-plans-v1_4/q76/simplified.txt | 30 +-- .../approved-plans-v1_4/q77/simplified.txt | 50 ++--- .../approved-plans-v1_4/q78/simplified.txt | 14 +- .../approved-plans-v1_4/q79/simplified.txt | 26 +-- .../approved-plans-v1_4/q8/simplified.txt | 26 +-- .../approved-plans-v1_4/q80/simplified.txt | 24 +-- .../approved-plans-v1_4/q81/simplified.txt | 28 +-- .../approved-plans-v1_4/q82/simplified.txt | 22 +-- .../approved-plans-v1_4/q83/simplified.txt | 44 ++--- .../approved-plans-v1_4/q84/simplified.txt | 28 +-- .../approved-plans-v1_4/q85/simplified.txt | 42 ++--- .../approved-plans-v1_4/q86/simplified.txt | 18 +- .../approved-plans-v1_4/q87/simplified.txt | 28 +-- .../approved-plans-v1_4/q88/simplified.txt | 104 +++++----- .../approved-plans-v1_4/q89/simplified.txt | 24 +-- .../approved-plans-v1_4/q9/simplified.txt | 20 +- .../approved-plans-v1_4/q90/simplified.txt | 32 ++-- .../approved-plans-v1_4/q91/simplified.txt | 42 ++--- .../approved-plans-v1_4/q92/simplified.txt | 20 +- .../approved-plans-v1_4/q93/simplified.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 8 +- .../approved-plans-v1_4/q95/simplified.txt | 12 +- .../approved-plans-v1_4/q96/simplified.txt | 20 +- .../approved-plans-v1_4/q97/simplified.txt | 10 +- .../approved-plans-v1_4/q98/simplified.txt | 18 +- .../approved-plans-v1_4/q99/simplified.txt | 28 +-- .../approved-plans-v2_7/q10a/simplified.txt | 38 ++-- .../approved-plans-v2_7/q11/simplified.txt | 60 +++--- .../approved-plans-v2_7/q12/simplified.txt | 18 +- .../approved-plans-v2_7/q14/simplified.txt | 58 +++--- .../approved-plans-v2_7/q14a/simplified.txt | 62 +++--- .../approved-plans-v2_7/q18a/simplified.txt | 118 ++++++------ .../approved-plans-v2_7/q20/simplified.txt | 18 +- .../approved-plans-v2_7/q22/simplified.txt | 16 +- .../approved-plans-v2_7/q22a/simplified.txt | 22 +-- .../approved-plans-v2_7/q24/simplified.txt | 14 +- .../approved-plans-v2_7/q27a/simplified.txt | 60 +++--- .../approved-plans-v2_7/q34/simplified.txt | 26 +-- .../approved-plans-v2_7/q35/simplified.txt | 22 +-- .../approved-plans-v2_7/q35a/simplified.txt | 38 ++-- .../approved-plans-v2_7/q36a/simplified.txt | 24 +-- .../approved-plans-v2_7/q47/simplified.txt | 22 +-- .../approved-plans-v2_7/q49/simplified.txt | 42 ++--- .../approved-plans-v2_7/q51a/simplified.txt | 18 +- .../approved-plans-v2_7/q57/simplified.txt | 22 +-- .../approved-plans-v2_7/q5a/simplified.txt | 58 +++--- .../approved-plans-v2_7/q6/simplified.txt | 30 +-- .../approved-plans-v2_7/q64/simplified.txt | 32 ++-- .../approved-plans-v2_7/q67a/simplified.txt | 24 +-- .../approved-plans-v2_7/q70a/simplified.txt | 24 +-- .../approved-plans-v2_7/q72/simplified.txt | 50 ++--- .../approved-plans-v2_7/q74/simplified.txt | 60 +++--- .../approved-plans-v2_7/q75/simplified.txt | 58 +++--- .../approved-plans-v2_7/q77a/simplified.txt | 50 ++--- .../approved-plans-v2_7/q78/simplified.txt | 14 +- .../approved-plans-v2_7/q80a/simplified.txt | 24 +-- .../approved-plans-v2_7/q86a/simplified.txt | 18 +- .../approved-plans-v2_7/q98/simplified.txt | 18 +- 145 files changed, 2285 insertions(+), 2312 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index e0a23e41e..fb1f62542 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -328,13 +328,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometProjectExec( + CometProjectExec( nativeOp, - op.output, + op, op.projectList, + op.output, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -343,9 +343,7 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometFilterExec(nativeOp, op.output, op.condition, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometFilterExec(nativeOp, op, op.condition, op.child, SerializedPlan(None)) case None => op } @@ -354,15 +352,7 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometSortExec( - nativeOp, - op.output, - op.outputOrdering, - op.sortOrder, - op.child, - SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometSortExec(nativeOp, op, op.sortOrder, op.child, SerializedPlan(None)) case None => op } @@ -371,9 +361,7 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometLocalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometLocalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) case None => op } @@ -382,9 +370,7 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometGlobalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometGlobalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) case None => op } @@ -396,9 +382,8 @@ class CometSparkSessionExtensions QueryPlanSerde.operator2Proto(op) match { case Some(nativeOp) => val offset = getOffset(op) - val newPlan = - CometCollectLimitExec(op.limit, offset, op.child) - val cometOp = setLogicalLink(newPlan, op) + val cometOp = + CometCollectLimitExec(op, op.limit, offset, op.child) CometSinkPlaceHolder(nativeOp, op, cometOp) case None => op @@ -408,28 +393,12 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometExpandExec( - nativeOp, - op.output, - op.projections, - op.child, - SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometExpandExec(nativeOp, op, op.projections, op.child, SerializedPlan(None)) case None => op } - case op @ HashAggregateExec( - _, - _, - _, - groupingExprs, - aggExprs, - _, - _, - resultExpressions, - child) => + case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) => val modes = aggExprs.map(_.mode).distinct if (!modes.isEmpty && modes.size != 1) { @@ -453,17 +422,15 @@ class CometSparkSessionExtensions // modes is empty too. If aggExprs is not empty, we need to verify all the // aggregates have the same mode. assert(modes.length == 1 || modes.length == 0) - val newPlan = CometHashAggregateExec( + CometHashAggregateExec( nativeOp, - op.output, + op, groupingExprs, aggExprs, - resultExpressions, child.output, if (modes.nonEmpty) Some(modes.head) else None, child, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -476,10 +443,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometHashJoinExec( + CometHashJoinExec( nativeOp, - op.output, - op.outputOrdering, + op, op.leftKeys, op.rightKeys, op.joinType, @@ -488,7 +454,6 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -510,10 +475,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometBroadcastHashJoinExec( + CometBroadcastHashJoinExec( nativeOp, - op.output, - op.outputOrdering, + op, op.leftKeys, op.rightKeys, op.joinType, @@ -522,7 +486,6 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -533,10 +496,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometSortMergeJoinExec( + CometSortMergeJoinExec( nativeOp, - op.output, - op.outputOrdering, + op, op.leftKeys, op.rightKeys, op.joinType, @@ -544,7 +506,6 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -574,8 +535,7 @@ class CometSparkSessionExtensions && isCometNative(child) => QueryPlanSerde.operator2Proto(c) match { case Some(nativeOp) => - val newPlan = CometCoalesceExec(c.output, numPartitions, child) - val cometOp = setLogicalLink(newPlan, c) + val cometOp = CometCoalesceExec(c, numPartitions, child) CometSinkPlaceHolder(nativeOp, c, cometOp) case None => c @@ -598,14 +558,8 @@ class CometSparkSessionExtensions CometTakeOrderedAndProjectExec.isSupported(s) => QueryPlanSerde.operator2Proto(s) match { case Some(nativeOp) => - val newPlan = - CometTakeOrderedAndProjectExec( - s.output, - s.limit, - s.sortOrder, - s.projectList, - s.child) - val cometOp = setLogicalLink(newPlan, s) + val cometOp = + CometTakeOrderedAndProjectExec(s, s.limit, s.sortOrder, s.projectList, s.child) CometSinkPlaceHolder(nativeOp, s, cometOp) case None => s @@ -625,14 +579,8 @@ class CometSparkSessionExtensions val newOp = transform1(w) newOp match { case Some(nativeOp) => - val newPlan = - CometWindowExec( - w.output, - w.windowExpression, - w.partitionSpec, - w.orderSpec, - w.child) - val cometOp = setLogicalLink(newPlan, w) + val cometOp = + CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) CometSinkPlaceHolder(nativeOp, w, cometOp) case None => w @@ -643,8 +591,7 @@ class CometSparkSessionExtensions u.children.forall(isCometNative) => QueryPlanSerde.operator2Proto(u) match { case Some(nativeOp) => - val newPlan = CometUnionExec(u.output, u.children) - val cometOp = setLogicalLink(newPlan, u) + val cometOp = CometUnionExec(u, u.children) CometSinkPlaceHolder(nativeOp, u, cometOp) case None => u @@ -684,8 +631,7 @@ class CometSparkSessionExtensions isSpark34Plus => // Spark 3.4+ only QueryPlanSerde.operator2Proto(b) match { case Some(nativeOp) => - val newPlan = CometBroadcastExchangeExec(b.output, b.child) - val cometOp = setLogicalLink(newPlan, b) + val cometOp = CometBroadcastExchangeExec(b, b.child) CometSinkPlaceHolder(nativeOp, b, cometOp) case None => b } @@ -876,6 +822,40 @@ class CometSparkSessionExtensions case CometScanWrapper(_, s) => s } + // Set up logical links + newPlan = newPlan.transform { + case op: CometExec => + if (op.originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + case op: CometShuffleExchangeExec => + // Original Spark shuffle exchange operator might have empty logical link. + // But the `setLogicalLink` call above on downstream operator of + // `CometShuffleExchangeExec` will set its logical link to the downstream + // operators which cause AQE behavior to be incorrect. So we need to unset + // the logical link here. + if (op.originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + + case op: CometBroadcastExchangeExec => + if (op.originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + } + // Convert native execution block by linking consecutive native operators. var firstNativeOp = true newPlan.transformDown { @@ -907,44 +887,6 @@ class CometSparkSessionExtensions }.flatten } - /** - * Set up logical links for transformed Comet operators. - */ - def setLogicalLink(newPlan: SparkPlan, originalPlan: SparkPlan): SparkPlan = { - newPlan match { - case op: CometExec => - if (originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - case op: CometShuffleExchangeExec => - // Original Spark shuffle exchange operator might have empty logical link. - // But the `setLogicalLink` call above on downstream operator of - // `CometShuffleExchangeExec` will set its logical link to the downstream - // operators which cause AQE behavior to be incorrect. So we need to unset - // the logical link here. - if (originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - - case op: CometBroadcastExchangeExec => - if (originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - } - } - /** * Returns true if a given spark plan is Comet shuffle operator. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index 426806916..38247b2c4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -31,7 +31,6 @@ import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} @@ -61,9 +60,7 @@ import org.apache.comet.CometRuntimeException * Note that this only supports Spark 3.4 and later, because the serialization class * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later. */ -case class CometBroadcastExchangeExec( - override val output: Seq[Attribute], - override val child: SparkPlan) +case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) extends BroadcastExchangeLike with ShimCometBroadcastExchangeExec { import CometBroadcastExchangeExec._ @@ -77,6 +74,10 @@ case class CometBroadcastExchangeExec( "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) + override def doCanonicalize(): SparkPlan = { + CometBroadcastExchangeExec(originalPlan.canonicalized, child.canonicalized) + } + override def runtimeStatistics: Statistics = { val dataSize = metrics("dataSize").value val rowCount = metrics("numOutputRows").value @@ -236,7 +237,7 @@ case class CometBroadcastExchangeExec( override def equals(obj: Any): Boolean = { obj match { case other: CometBroadcastExchangeExec => - this.output == other.output && + this.originalPlan == other.originalPlan && this.child == other.child case _ => false diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala index 1f656b117..cc635d739 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -32,7 +31,7 @@ import com.google.common.base.Objects * more efficient when including it in a Comet query plan. */ case class CometCoalesceExec( - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, numPartitions: Int, child: SparkPlan) extends CometExec diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index 0090c537c..d2c9158ee 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -22,8 +22,6 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer} @@ -40,11 +38,13 @@ import com.google.common.base.Objects * * TODO: support offset semantics */ -case class CometCollectLimitExec(limit: Int, offset: Int, child: SparkPlan) +case class CometCollectLimitExec( + override val originalPlan: SparkPlan, + limit: Int, + offset: Int, + child: SparkPlan) extends CometExec with UnaryExecNode { - override def output: Seq[Attribute] = child.output - override def outputPartitioning: Partitioning = SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 2f5121bed..6e9bfe424 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -41,13 +41,15 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec * contains two native executions separated by a Comet shuffle exchange. */ case class CometTakeOrderedAndProjectExec( - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, limit: Int, sortOrder: Seq[SortOrder], projectList: Seq[NamedExpression], child: SparkPlan) extends CometExec with UnaryExecNode { + override def output: Seq[Attribute] = projectList.map(_.toAttribute) + private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 221b725d5..9685e75e1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -42,7 +42,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi * executions separated by a Comet shuffle exchange. */ case class CometWindowExec( - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], @@ -52,6 +52,8 @@ case class CometWindowExec( override def nodeName: String = "CometWindowExec" + override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) + private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 5d4848216..cec64d7a8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -64,6 +64,7 @@ import org.apache.comet.shims.ShimCometShuffleExchangeExec case class CometShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, + originalPlan: ShuffleExchangeLike, shuffleOrigin: ShuffleOrigin = ENSURE_REQUIREMENTS, shuffleType: ShuffleType = CometNativeShuffle, advisoryPartitionSize: Option[Long] = None) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 732213158..8b1ba168f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf @@ -56,15 +57,28 @@ import org.apache.comet.shims.ShimCometBroadcastHashJoinExec */ abstract class CometExec extends CometPlan { + /** The original Spark operator from which this Comet operator is converted from */ + def originalPlan: SparkPlan + /** Comet always support columnar execution */ override def supportsColumnar: Boolean = true + override def output: Seq[Attribute] = originalPlan.output + override def doExecute(): RDD[InternalRow] = ColumnarToRowExec(this).doExecute() override def executeCollect(): Array[InternalRow] = ColumnarToRowExec(this).executeCollect() + override def outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering + + // `CometExec` reuses the outputPartitioning of the original SparkPlan. + // Note that if the outputPartitioning of the original SparkPlan depends on its children, + // we should override this method in the specific CometExec, because Spark AQE may change the + // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. + override def outputPartitioning: Partitioning = originalPlan.outputPartitioning + /** * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. */ @@ -140,7 +154,7 @@ abstract class CometNativeExec extends CometExec { /** The Comet native operator */ def nativeOp: Operator - override protected def doPrepare(): Unit = prepareSubqueries(this) + override protected def doPrepare(): Unit = prepareSubqueries(originalPlan) override lazy val metrics: Map[String, SQLMetric] = CometMetricNode.baselineMetrics(sparkContext) @@ -203,12 +217,12 @@ abstract class CometNativeExec extends CometExec { val it = new CometExecIterator(CometExec.newIterId, inputs, serializedPlanCopy, nativeMetrics) - setSubqueries(it.id, this) + setSubqueries(it.id, originalPlan) Option(TaskContext.get()).foreach { context => context.addTaskCompletionListener[Unit] { _ => it.close() - cleanSubqueries(it.id, this) + cleanSubqueries(it.id, originalPlan) } } @@ -349,7 +363,10 @@ abstract class CometNativeExec extends CometExec { } override protected def doCanonicalize(): SparkPlan = { - val canonicalizedPlan = super.doCanonicalize().asInstanceOf[CometNativeExec] + val canonicalizedPlan = super + .doCanonicalize() + .asInstanceOf[CometNativeExec] + .canonicalizePlans() if (serializedPlanOpt.isDefined) { // If the plan is a boundary node, we should remove the serialized plan. @@ -358,6 +375,24 @@ abstract class CometNativeExec extends CometExec { canonicalizedPlan } } + + /** + * Canonicalizes the plans of Product parameters in Comet native operators. + */ + protected def canonicalizePlans(): CometNativeExec = { + def transform(arg: Any): AnyRef = arg match { + case sparkPlan: SparkPlan if !sparkPlan.isInstanceOf[CometNativeExec] => + // Different to Spark, Comet native query node might have a Spark plan as Product element. + // We need to canonicalize the Spark plan. But it cannot be another Comet native query node, + // otherwise it will cause recursive canonicalization. + sparkPlan.canonicalized + case other: AnyRef => other + case null => null + } + + val newArgs = mapProductIterator(transform) + makeCopy(newArgs).asInstanceOf[CometNativeExec] + } } abstract class CometUnaryExec extends CometNativeExec with UnaryExecNode @@ -376,8 +411,9 @@ case class SerializedPlan(plan: Option[Array[Byte]]) { case class CometProjectExec( override val nativeOp: Operator, - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, projectList: Seq[NamedExpression], + override val output: Seq[Attribute], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec @@ -406,20 +442,19 @@ case class CometProjectExec( case class CometFilterExec( override val nativeOp: Operator, - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, condition: Expression, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { override def outputPartitioning: Partitioning = child.outputPartitioning - override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(output, condition, child) + Iterator(originalPlan.output, condition, child) override def equals(obj: Any): Boolean = { obj match { @@ -444,8 +479,7 @@ case class CometFilterExec( case class CometSortExec( override val nativeOp: Operator, - override val output: Seq[Attribute], - override val outputOrdering: Seq[SortOrder], + override val originalPlan: SparkPlan, sortOrder: Seq[SortOrder], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -457,7 +491,7 @@ case class CometSortExec( this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(output, sortOrder, child) + Iterator(originalPlan.output, sortOrder, child) override def equals(obj: Any): Boolean = { obj match { @@ -480,14 +514,13 @@ case class CometSortExec( case class CometLocalLimitExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning - override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -511,14 +544,13 @@ case class CometLocalLimitExec( case class CometGlobalLimitExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning - override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -540,13 +572,11 @@ case class CometGlobalLimitExec( case class CometExpandExec( override val nativeOp: Operator, - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, projections: Seq[Seq[Expression]], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { - override def outputPartitioning: Partitioning = UnknownPartitioning(0) - override def producedAttributes: AttributeSet = outputSet override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -570,7 +600,7 @@ case class CometExpandExec( override lazy val metrics: Map[String, SQLMetric] = Map.empty } -case class CometUnionExec(override val output: Seq[Attribute], children: Seq[SparkPlan]) +case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan]) extends CometExec { override def doExecuteColumnar(): RDD[ColumnarBatch] = { sparkContext.union(children.map(_.executeColumnar())) @@ -603,10 +633,9 @@ case class CometUnionExec(override val output: Seq[Attribute], children: Seq[Spa case class CometHashAggregateExec( override val nativeOp: Operator, - override val output: Seq[Attribute], + override val originalPlan: SparkPlan, groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], - resultExpressions: Seq[NamedExpression], input: Seq[Attribute], mode: Option[AggregateMode], child: SparkPlan, @@ -645,13 +674,13 @@ case class CometHashAggregateExec( override def hashCode(): Int = Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child) - override protected def outputExpressions: Seq[NamedExpression] = resultExpressions + override protected def outputExpressions: Seq[NamedExpression] = + originalPlan.asInstanceOf[HashAggregateExec].resultExpressions } case class CometHashJoinExec( override val nativeOp: Operator, - override val output: Seq[Attribute], - override val outputOrdering: Seq[SortOrder], + override val originalPlan: SparkPlan, leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -703,8 +732,7 @@ case class CometHashJoinExec( case class CometBroadcastHashJoinExec( override val nativeOp: Operator, - override val output: Seq[Attribute], - override val outputOrdering: Seq[SortOrder], + override val originalPlan: SparkPlan, leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -838,8 +866,7 @@ case class CometBroadcastHashJoinExec( case class CometSortMergeJoinExec( override val nativeOp: Operator, - override val output: Seq[Attribute], - override val outputOrdering: Seq[SortOrder], + override val originalPlan: SparkPlan, leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -894,10 +921,9 @@ case class CometSortMergeJoinExec( "join_time" -> SQLMetrics.createNanoTimingMetric(sparkContext, "Total time for joining")) } -case class CometScanWrapper(override val nativeOp: Operator, originalPlan: SparkPlan) +case class CometScanWrapper(override val nativeOp: Operator, override val originalPlan: SparkPlan) extends CometNativeExec with LeafExecNode { - override def output: Seq[Attribute] = originalPlan.output override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override def stringArgs: Iterator[Any] = Iterator(originalPlan.output, originalPlan) } @@ -911,11 +937,9 @@ case class CometScanWrapper(override val nativeOp: Operator, originalPlan: Spark */ case class CometSinkPlaceHolder( override val nativeOp: Operator, // Must be a Scan - originalPlan: SparkPlan, + override val originalPlan: SparkPlan, child: SparkPlan) extends CometUnaryExec { - override def output: Seq[Attribute] = originalPlan.output - override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 9fadcc680..965b6851e 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -35,6 +35,7 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, + s, s.shuffleOrigin, shuffleType, advisoryPartitionSize) diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 2be852ae5..559e327b4 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -31,6 +31,7 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, + s, s.shuffleOrigin, shuffleType, s.advisoryPartitionSize) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 238f9b702..000160518 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_customer_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] + CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_customer_id] + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index efd4b187d..19243e359 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 + CometBroadcastExchange #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] + CometFilter [ca_county,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 127a5a291..dd1a52206 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastExchange #7 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 4c2d3f3d7..905a35c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 98d995db0..4de403664 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] - CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 + CometBroadcastExchange #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] + CometFilter [ca_country,ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 + CometBroadcastExchange #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 + CometBroadcastExchange #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 2257d398f..e3dfa631b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] + CometHashAggregate [quantity,list_price] + CometUnion CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #8 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastExchange #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 + CometBroadcastExchange #11 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index aa49638c7..799f74a36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] + CometHashAggregate [quantity,list_price] + CometUnion CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] + CometFilter [d_week_seq,d_date_sk] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometFilter [d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #7 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastExchange #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #9 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 + CometBroadcastExchange #10 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] + CometFilter [d_week_seq,d_date_sk] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometFilter [d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index b876fe4de..a03346372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,sum,cs_sales_price] + CometHashAggregate [ca_zip,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometBroadcastExchange #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometFilter [ca_address_sk,ca_state,ca_zip] + CometBroadcastExchange #4 + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index 8935abb54..a55c182be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] + CometFilter [cc_county,cc_call_center_sk] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index ae6bab279..77aba376e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [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,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] + CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] + CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] + CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 + CometBroadcastExchange #7 CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] + CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] + CometBroadcastExchange #8 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometFilter [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #9 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 2519d58a2..3d101857b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometBroadcastExchange #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometBroadcastExchange #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 + CometBroadcastExchange #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange #6 + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 + CometBroadcastExchange #7 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #8 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 163a31f47..18a69bcb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometBroadcastExchange #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometFilter [i_manager_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometBroadcastExchange #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometBroadcastExchange #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange #6 + CometFilter [s_zip,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 5d0658192..c7999d981 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] + CometHashAggregate [d_week_seq,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] - CometUnion [sold_date_sk,sales_price] + CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometUnion CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometFilter [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange #3 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] + CometFilter [d_year,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] + CometFilter [d_year,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 08088a386..d805e3868 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index bd81a3a7b..52bd7a85e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometFilter [inv_warehouse_sk,inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #3 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 + CometBroadcastExchange #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] + CometFilter [i_current_price,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] + CometBroadcastExchange #5 + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index 7d36dc400..bda583c17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #4 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 + CometBroadcastExchange #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index e065a0a87..f818fd25f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometHashAggregate [_groupingexpression,i_item_sk,d_date] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 + CometBroadcastExchange #7 CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] + CometBroadcastExchange #8 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 + CometBroadcastExchange #14 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 + CometBroadcastExchange #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 2862f64cc..54ee3dbde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] + CometHashAggregate [_groupingexpression,i_item_sk,d_date] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 + CometBroadcastExchange #7 CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] + CometBroadcastExchange #8 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 + CometBroadcastExchange #14 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 + CometBroadcastExchange #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 30091beb8..8ebd45fd1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_zip,ca_country] + CometFilter [ca_country,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 30091beb8..8ebd45fd1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_zip,ca_country] + CometFilter [ca_country,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 1f4faaac5..74a5e8777 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 + CometBroadcastExchange #7 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometFilter [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange #8 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometFilter [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #9 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index a39fbbb79..cba306f68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 + CometBroadcastExchange #3 CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #5 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 + CometBroadcastExchange #6 CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 9c4dd659a..7fcbe967a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 + CometBroadcastExchange #3 CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometFilter [s_store_sk,s_state] + CometBroadcastExchange #5 + CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 80c886708..47a4fe2c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price,sum,count,count] + CometHashAggregate [ss_list_price] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price,sum,count,count] + CometHashAggregate [ss_list_price] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price,sum,count,count] + CometHashAggregate [ss_list_price] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price,sum,count,count] + CometHashAggregate [ss_list_price] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price,sum,count,count] + CometHashAggregate [ss_list_price] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price,sum,count,count] + CometHashAggregate [ss_list_price] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 88b4ecd1d..2bcef6168 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.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 #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 + CometBroadcastExchange #7 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 + CometBroadcastExchange #8 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 + CometBroadcastExchange #9 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometFilter [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange #10 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometFilter [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #11 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index bccda58a4..cc9c4edf3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometBroadcastExchange #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometFilter [i_manufact_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 60cad82dd..50472502e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_addr_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index a48c7ad26..a94a8a94d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #9 + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #13 + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index c3bfdfd77..b8df1e929 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 + CometBroadcastExchange #3 CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] + CometFilter [i_manufact_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 7a3c2c153..af2b7cb5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 + CometBroadcastExchange #5 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] + CometFilter [ca_gmt_offset,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_manufact_id] #6 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_manufact_id,i_manufact_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 + CometBroadcastExchange #7 CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] + CometFilter [i_category] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index e1f7f9bd2..eefd38343 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometHashAggregate [ss_ticket_number,ss_customer_sk] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 + CometBroadcastExchange #5 CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] + CometFilter [s_county,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 + CometBroadcastExchange #6 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 9deb51342..efe0b0b4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 + CometBroadcastExchange #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_year,d_qoy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_year,d_qoy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 4ad9b501b..574a20fc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] + CometBroadcastExchange #5 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 + CometBroadcastExchange #6 CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] + CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index efd86d5ab..c3fcd79f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometBroadcastHashJoin [i_item_sk,cs_item_sk] + CometBroadcastExchange #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometBroadcastHashJoin [i_item_sk,inv_item_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometFilter [i_current_price,i_manufact_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometBroadcastExchange #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 9d667265c..cfac83844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 + CometBroadcastExchange #4 CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometFilter [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange #5 + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 0c709e4f7..7d28d5c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 + CometBroadcastExchange #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 + CometBroadcastExchange #6 CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 + CometBroadcastExchange #10 CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 0c709e4f7..7d28d5c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 + CometBroadcastExchange #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 + CometBroadcastExchange #6 CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange [d_date_sk,d_moy] #10 + CometBroadcastExchange #10 CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 1a19c58ff..048da153b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastExchange #7 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometBroadcastExchange #12 + CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 - CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometBroadcastExchange #15 + CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange #18 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange #21 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 11c0201a8..10e0735b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_warehouse_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk,w_state] + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] + CometFilter [i_current_price,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 457d441ce..e31217066 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact,count] + CometHashAggregate [i_manufact] CometProject [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometFilter [i_category,i_color,i_units,i_size,i_manufact] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index e3b850ef5..3e69a3341 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometBroadcastExchange #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + CometFilter [i_manager_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index 5a060b85c..d29a65bd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometBroadcastExchange #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometFilter [s_gmt_offset,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index f713176f1..701a90912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -27,9 +27,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_store_sk,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_store_sk,ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -37,9 +37,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_item_sk,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -57,7 +57,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_product_name] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index f60fdb18a..c376c4fcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometFilter [ws_bill_customer_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 + CometBroadcastExchange #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometFilter [ca_address_sk,ca_city,ca_zip] + CometBroadcastExchange #4 + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk,i_item_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 57defd961..93e3eb05b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] + CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] + CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 + CometBroadcastExchange #4 CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] + CometFilter [s_city,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 + CometBroadcastExchange #5 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometBroadcastExchange #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index b6e5c469b..d2e615f39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometBroadcastExchange #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 5f628dbae..6ebf6af07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,ss_quantity] + CometHashAggregate [ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] - CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 + CometBroadcastExchange #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 + CometBroadcastExchange #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 + CometBroadcastExchange #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] + CometFilter [ca_country,ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 0e6b65b06..8d7b158d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 537cba446..c4ea8fe24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometBroadcastHashJoin [store_sk,s_store_sk] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] + CometUnion CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] + CometBroadcastExchange #5 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] + CometUnion CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometBroadcastExchange #7 + CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] + CometUnion CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - CometFilter [web_site_sk,web_site_id] + CometBroadcastExchange #10 + CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 58adabcec..473b9cdd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] + CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 - CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange #2 + CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 - CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastExchange #4 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index 988297f02..ce6005da6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 + CometBroadcastExchange #6 CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index a0932c7cc..557dd3b4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometBroadcastExchange #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometFilter [i_manager_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index b2199fb34..83e53bb66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometBroadcastExchange #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 + CometBroadcastExchange #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index 2193d875c..c6886735f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] - CometUnion [sold_date_sk,customer_sk,item_sk] + CometBroadcastHashJoin [item_sk,i_item_sk] + CometUnion CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 + CometBroadcastExchange #5 CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] + CometFilter [i_category,i_class,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 + CometBroadcastExchange #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] + CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] + CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index fc38884c1..2750a6ba2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometBroadcastExchange #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometFilter [i_manager_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 0b48046fd..7fdead831 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [i_item_id,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 + CometBroadcastExchange #5 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] + CometFilter [ca_gmt_offset,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_item_id,i_item_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 + CometBroadcastExchange #7 CometProject [i_item_id] - CometFilter [i_item_id,i_color] + CometFilter [i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [i_item_id,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [i_item_id,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index fb2b09b9d..c630cad48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] + CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [cs_item_sk,cs_call_center_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometBroadcastExchange #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index fc7c21e94..2ed2bde44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [i_item_id,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 + CometBroadcastExchange #3 CometProject [d_date] - CometFilter [d_date,d_week_seq] + CometFilter [d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] + CometFilter [d_date] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 + CometBroadcastExchange #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [i_item_id,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [i_item_id,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 6c1e6cd92..e00d52dbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometFilter [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange #2 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 824670101..89a080d85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #2 + CometFilter [c_current_addr_sk,c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] + CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] + CometFilter [i_current_price,i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,sum,count,i_current_price] - CometFilter [i_current_price,i_category] + CometHashAggregate [i_category,i_current_price] + CometFilter [i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 7cfcb75da..b76e7c9b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [i_item_id,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 + CometBroadcastExchange #5 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] + CometFilter [ca_gmt_offset,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_item_id,i_item_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 + CometBroadcastExchange #7 CometProject [i_item_id] - CometFilter [i_item_id,i_category] + CometFilter [i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [i_item_id,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [i_item_id,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 1b2af33b0..4ca1dd667 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 + CometBroadcastExchange #3 CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] + CometFilter [s_gmt_offset,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 + CometBroadcastExchange #4 CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 + CometBroadcastExchange #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 + CometBroadcastExchange #7 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] + CometFilter [ca_gmt_offset,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 + CometBroadcastExchange #8 CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index 0b4fc61a1..c6b7e1834 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #2 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] + CometBroadcastExchange #3 + CometFilter [sm_ship_mode_sk] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] + CometBroadcastExchange #4 + CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index b2033c7a9..35e09ec7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 + CometBroadcastExchange #5 CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 + CometBroadcastExchange #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index 3b5e4f19e..d972e0082 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [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] #4 - CometFilter [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [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] #20 - CometFilter [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 6cb247973..8de564ed1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_name] + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 72133811b..d746739b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastExchange #4 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 + CometBroadcastExchange #6 CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] + CometFilter [t_time,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 + CometBroadcastExchange #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] + CometFilter [sm_carrier,sm_ship_mode_sk] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index c37569802..b529cb5ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,15 +30,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometBroadcastExchange #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] + CometBroadcastExchange #5 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index c5f56a69c..43f44c9f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 + CometBroadcastExchange #4 CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] + CometFilter [s_city,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 + CometBroadcastExchange #5 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 + CometBroadcastExchange #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index c9cc4959b..de2d5eeda 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 + CometBroadcastExchange #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index f327d5b06..c583ba8e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 + CometBroadcastExchange #3 CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #5 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 + CometBroadcastExchange #6 CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index f95e8d040..f4a03a2d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -57,15 +57,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [s_state,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometFilter [s_store_sk,s_state] + CometBroadcastExchange #8 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index c604a8fa7..fd6777886 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] + CometBroadcastHashJoin [time_sk,t_time_sk] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometBroadcastHashJoin [i_item_sk,sold_item_sk] + CometBroadcastExchange #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometFilter [i_manager_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion [ext_price,sold_item_sk,time_sk] + CometUnion CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_sold_time_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_sold_time_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_sold_time_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 + CometBroadcastExchange #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + CometFilter [t_meal_time,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 6cba2d0e6..bea1fd4a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] + CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] + CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] + CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange #4 + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 + CometBroadcastExchange #7 CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] + CometFilter [cd_marital_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 + CometBroadcastExchange #8 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] + CometFilter [hd_buy_potential,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometBroadcastExchange #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange #10 + CometFilter [d_week_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index c91c4cf47..060c3e153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometHashAggregate [ss_ticket_number,ss_customer_sk] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 + CometBroadcastExchange #5 CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] + CometFilter [s_county,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 + CometBroadcastExchange #6 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 7c2a42ca3..2e6286d93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometBroadcastExchange #7 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 27a4dcb67..34866bc37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #7 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #18 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index e463296d4..473eef9e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] - CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometUnion CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometFilter [i_item_sk,i_category] + CometBroadcastExchange #2 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #3 + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_ship_customer_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_ship_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index f9088f784..590c59fdc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 + CometBroadcastExchange #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [wp_web_page_sk] #12 + CometBroadcastExchange #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 13a8aecff..280687e30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index 57598ec35..b68a9474b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] + CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] + CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 + CometBroadcastExchange #4 CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] + CometFilter [s_number_employees,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 + CometBroadcastExchange #5 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 637f3b7ba..adf555417 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometFilter [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange #4 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,count] + CometHashAggregate [ca_zip] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometFilter [ca_address_sk,ca_zip] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 + CometBroadcastExchange #9 CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] + CometFilter [c_preferred_cust_flag,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index cdc8dc486..7e257bdc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] + CometFilter [i_current_price,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] + CometFilter [p_channel_tv,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometFilter [wr_item_sk,wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk,web_site_id] + CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index e576a079b..f80554fb5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_addr_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 71a1c8e17..a4d96e6f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 + CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastExchange #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] + CometBroadcastHashJoin [i_item_sk,inv_item_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometFilter [i_current_price,i_manufact_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 + CometBroadcastExchange #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 390c20b7f..ef7d35e21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,sr_return_quantity] + CometHashAggregate [i_item_id,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + CometBroadcastHashJoin [sr_item_sk,i_item_sk] + CometFilter [sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 + CometBroadcastExchange #3 CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 + CometBroadcastExchange #4 CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] + CometFilter [d_date] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometBroadcastExchange #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,cr_return_quantity] + CometHashAggregate [i_item_id,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + CometBroadcastHashJoin [cr_item_sk,i_item_sk] + CometFilter [cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,wr_return_quantity] + CometHashAggregate [i_item_id,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] - CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + CometBroadcastHashJoin [wr_item_sk,i_item_sk] + CometFilter [wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index 646285a08..fe7f7a207 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 + CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] + CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange [ca_address_sk] #2 + CometBroadcastExchange #2 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] + CometFilter [ca_city,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 + CometBroadcastExchange #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 + CometBroadcastExchange #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 + CometBroadcastExchange #5 CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] + CometFilter [sr_cdemo_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index 2313467ae..ecae29c78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] + CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] + CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 - CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + CometBroadcastExchange #2 + CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] + CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange [wp_web_page_sk] #4 + CometBroadcastExchange #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 + CometBroadcastExchange #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 + CometBroadcastExchange #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 + CometBroadcastExchange #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] + CometFilter [ca_country,ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 + CometBroadcastExchange #8 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometFilter [r_reason_sk,r_reason_desc] + CometBroadcastExchange #9 + CometFilter [r_reason_sk] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index a6d1c25fc..4218938c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometFilter [i_item_sk,i_class,i_category] + CometBroadcastExchange #5 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 9d667265c..cfac83844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 + CometBroadcastExchange #4 CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometFilter [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange #5 + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index d3af10d19..b846d25d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 + CometBroadcastExchange #2 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 + CometBroadcastExchange #3 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 + CometBroadcastExchange #4 CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] + CometFilter [s_store_name,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 + CometBroadcastExchange #7 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 + CometBroadcastExchange #10 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange #13 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange #16 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange #19 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange #22 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange #25 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index aa858c158..507ac8a91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_class,i_category] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometFilter [i_category,i_class,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 + CometBroadcastExchange #5 CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #6 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 5cb600551..c54606f6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_quantity] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_quantity] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_quantity] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_quantity] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_quantity] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 44159cc73..50c8494fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 + CometBroadcastExchange #2 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] + CometFilter [hd_dep_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 + CometBroadcastExchange #3 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] + CometFilter [t_hour,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 + CometBroadcastExchange #4 CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] + CometFilter [wp_char_count,wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 + CometBroadcastExchange #7 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] + CometFilter [t_hour,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index 232c174cd..b415eb5c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] + CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] + CometFilter [cc_call_center_sk] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 - CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometBroadcastExchange #3 + CometFilter [cr_call_center_sk,cr_returning_customer_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastExchange #6 + CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 + CometBroadcastExchange #7 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] + CometFilter [ca_gmt_offset,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #8 + CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 + CometBroadcastExchange #9 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] + CometFilter [hd_buy_potential,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 0681a64bd..e2f498028 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 + CometBroadcastExchange #3 CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] + CometFilter [i_manufact_id,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index a8eb231b7..3ec7ac7b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] + CometFilter [r_reason_desc,r_reason_sk] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index d54c9e0c9..34ddde768 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] + CometFilter [web_company_name,web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 178c25f66..5b699890c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometFilter [ws_order_number,ws_warehouse_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] + CometFilter [wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] + CometFilter [web_company_name,web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index e9d33a7f6..614915226 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [count] + CometHashAggregate CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 + CometBroadcastExchange #2 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] + CometFilter [hd_dep_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 + CometBroadcastExchange #3 CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] + CometFilter [t_hour,t_minute,t_time_sk] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 + CometBroadcastExchange #4 CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] + CometFilter [s_store_name,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 41f3e579f..24e6dceef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 3895cdc78..c03b8be9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index fc63929bb..adfe90bab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #2 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometFilter [sm_ship_mode_sk,sm_type] + CometBroadcastExchange #3 + CometFilter [sm_ship_mode_sk] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] + CometBroadcastExchange #4 + CometFilter [cc_call_center_sk] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 22c208f2a..520edc88d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 + CometBroadcastExchange #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] + CometBroadcastExchange #5 + CometUnion CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [ca_address_sk] #6 + CometBroadcastExchange #6 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] + CometFilter [ca_county,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastExchange #7 + CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 25010c5eb..ecc421bd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastExchange #7 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index a8b8a9cce..545f0ecec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index aa49638c7..799f74a36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] + CometHashAggregate [quantity,list_price] + CometUnion CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] + CometFilter [d_week_seq,d_date_sk] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometFilter [d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #7 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastExchange #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #9 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 + CometBroadcastExchange #10 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] + CometFilter [d_week_seq,d_date_sk] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometFilter [d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 7ef901a39..45061c290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,quantity,list_price] - CometUnion [quantity,list_price] + CometHashAggregate [quantity,list_price] + CometUnion CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #17 + CometBroadcastExchange #17 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #9 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 + CometBroadcastExchange #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #11 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 + CometBroadcastExchange #12 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index f854b2b31..d4deedf0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometBroadcastExchange #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometBroadcastExchange #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 + CometBroadcastExchange #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange #6 + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 + CometBroadcastExchange #7 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #8 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 - CometFilter [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #10 + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk,ca_country] #12 + CometBroadcastExchange #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk] #14 + CometBroadcastExchange #14 CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 + CometBroadcastExchange #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index e460ce5c9..513e6f979 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 0e864ab70..415b430f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometFilter [inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #4 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 30307e528..6ab6c4fd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 + CometBroadcastExchange #3 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #4 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 + CometBroadcastExchange #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 5b658f1d7..7024f439f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_country,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index de75d46ee..371254b08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 + CometBroadcastExchange #3 CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometFilter [s_store_sk,s_state] + CometBroadcastExchange #5 + CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometFilter [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [s_store_sk] #8 + CometBroadcastExchange #8 CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] + CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange [i_item_sk] #10 + CometBroadcastExchange #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 814e543ed..970562e5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometHashAggregate [ss_ticket_number,ss_customer_sk] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] + CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 + CometBroadcastExchange #5 CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] + CometFilter [s_county,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 + CometBroadcastExchange #6 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 0b55e23ab..a6b4add1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 + CometBroadcastExchange #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_year,d_qoy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_year,d_qoy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_state] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index 8d73022ee..e3a91e471 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 + CometBroadcastExchange #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_year,d_qoy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 + CometBroadcastExchange #4 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] + CometFilter [d_year,d_qoy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] + CometBroadcastExchange #5 + CometUnion CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [ca_address_sk,ca_state] #6 - CometFilter [ca_address_sk,ca_state] + CometBroadcastExchange #6 + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastExchange #7 + CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 256fddfc8..9c35ee397 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 + CometBroadcastExchange #7 CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] + CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index eeeb5ba36..a85302cc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometBroadcastExchange #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 0e6b65b06..8d7b158d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index e9c4d46ca..78dd29c2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 + CometBroadcastExchange #7 CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 968f7e1af..c2c8e089f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometFilter [i_item_sk,i_brand,i_category] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] + CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [cs_item_sk,cs_call_center_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year,d_moy] + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometBroadcastExchange #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index 1428c7ba5..dd66c9582 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometBroadcastHashJoin [store_sk,s_store_sk] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] + CometUnion CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometFilter [s_store_sk,s_store_id] + CometBroadcastExchange #6 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] + CometUnion CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometBroadcastExchange #8 + CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] + CometUnion CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometFilter [web_site_sk,web_site_id] + CometBroadcastExchange #11 + CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index cfb74b3b2..9607ab887 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometFilter [ca_address_sk,ca_state] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #2 + CometFilter [c_current_addr_sk,c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometBroadcastExchange #3 + CometFilter [ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] + CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] + CometFilter [i_current_price,i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,sum,count,i_current_price] - CometFilter [i_current_price,i_category] + CometHashAggregate [i_category,i_current_price] + CometFilter [i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 512a74f9f..2a0bc5bce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [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] #4 - CometFilter [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [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] #20 - CometFilter [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] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index dfa9ae66d..a3a99e312 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,17 +30,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometBroadcastExchange #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometFilter [s_store_sk,s_store_id] + CometBroadcastExchange #5 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index feca96b18..9e72edbb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -64,16 +64,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [s_state,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometFilter [s_store_sk,s_state] + CometBroadcastExchange #9 + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 6cba2d0e6..bea1fd4a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] + CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] + CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [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] + CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange #4 + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 + CometBroadcastExchange #7 CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] + CometFilter [cd_marital_status,cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 + CometBroadcastExchange #8 CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] + CometFilter [hd_buy_potential,hd_demo_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometBroadcastExchange #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange #10 + CometFilter [d_week_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index e5cf23f50..283fe5bde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometBroadcastExchange #7 + CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index 2aecc9e98..fb78d64b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #7 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] + CometBroadcastExchange #18 + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 34972d290..752b8c854 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 + CometBroadcastExchange #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [wp_web_page_sk] #13 + CometBroadcastExchange #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index b8e6e0605..49bd173f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] + CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index 0793067df..34e47dcba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometFilter [sr_item_sk,sr_ticket_number] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] + CometFilter [i_current_price,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] + CometFilter [p_channel_tv,p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometFilter [cr_item_sk,cr_order_number] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometFilter [wr_item_sk,wr_order_number] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk,web_site_id] + CometFilter [web_site_sk] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 5adee59be..0b7ad4726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,sum,ws_net_paid] + CometHashAggregate [i_category,i_class,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 + CometBroadcastExchange #5 CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] + CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometFilter [i_item_sk,i_class,i_category] + CometBroadcastExchange #6 + CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 64b162608..6484c2dcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometFilter [i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 + CometBroadcastExchange #6 CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] + CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] From 7e25818075f70a9c07b80a9f0869f7725b6cd622 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 7 Jul 2024 15:25:26 -0700 Subject: [PATCH 3/9] More --- .../main/scala/org/apache/spark/sql/comet/operators.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 8b1ba168f..ca3f69cdd 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -381,11 +381,13 @@ abstract class CometNativeExec extends CometExec { */ protected def canonicalizePlans(): CometNativeExec = { def transform(arg: Any): AnyRef = arg match { - case sparkPlan: SparkPlan if !sparkPlan.isInstanceOf[CometNativeExec] => + case sparkPlan: SparkPlan + if !sparkPlan.isInstanceOf[CometNativeExec] && + children.forall(_ != sparkPlan) => // Different to Spark, Comet native query node might have a Spark plan as Product element. // We need to canonicalize the Spark plan. But it cannot be another Comet native query node, // otherwise it will cause recursive canonicalization. - sparkPlan.canonicalized + null case other: AnyRef => other case null => null } From 6119cf12e492101e58901bbd6f5d7e4c239fcf69 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 10:23:40 -0700 Subject: [PATCH 4/9] Revert "Revert "fix: Remove original plan parameter from CometNativeExec"" This reverts commit 722dc07c75407edbf671d47f23769d4e543dbab5. --- .../comet/CometSparkSessionExtensions.scala | 178 ++++++++++++------ .../comet/CometBroadcastExchangeExec.scala | 11 +- .../spark/sql/comet/CometCoalesceExec.scala | 3 +- .../sql/comet/CometCollectLimitExec.scala | 10 +- .../CometTakeOrderedAndProjectExec.scala | 4 +- .../spark/sql/comet/CometWindowExec.scala | 4 +- .../shuffle/CometShuffleExchangeExec.scala | 1 - .../apache/spark/sql/comet/operators.scala | 70 ++++--- .../shims/ShimCometShuffleExchangeExec.scala | 1 - .../shims/ShimCometShuffleExchangeExec.scala | 1 - .../approved-plans-v1_4/q1/simplified.txt | 22 +-- .../approved-plans-v1_4/q10/simplified.txt | 22 +-- .../approved-plans-v1_4/q11/simplified.txt | 60 +++--- .../approved-plans-v1_4/q12/simplified.txt | 18 +- .../approved-plans-v1_4/q13/simplified.txt | 30 +-- .../approved-plans-v1_4/q14a/simplified.txt | 56 +++--- .../approved-plans-v1_4/q14b/simplified.txt | 58 +++--- .../approved-plans-v1_4/q15/simplified.txt | 22 +-- .../approved-plans-v1_4/q16/simplified.txt | 8 +- .../approved-plans-v1_4/q17/simplified.txt | 46 ++--- .../approved-plans-v1_4/q18/simplified.txt | 40 ++-- .../approved-plans-v1_4/q19/simplified.txt | 30 +-- .../approved-plans-v1_4/q2/simplified.txt | 14 +- .../approved-plans-v1_4/q20/simplified.txt | 18 +- .../approved-plans-v1_4/q21/simplified.txt | 24 +-- .../approved-plans-v1_4/q22/simplified.txt | 22 +-- .../approved-plans-v1_4/q23a/simplified.txt | 42 ++--- .../approved-plans-v1_4/q23b/simplified.txt | 48 ++--- .../approved-plans-v1_4/q24a/simplified.txt | 14 +- .../approved-plans-v1_4/q24b/simplified.txt | 14 +- .../approved-plans-v1_4/q25/simplified.txt | 46 ++--- .../approved-plans-v1_4/q26/simplified.txt | 30 +-- .../approved-plans-v1_4/q27/simplified.txt | 30 +-- .../approved-plans-v1_4/q28/simplified.txt | 24 +-- .../approved-plans-v1_4/q29/simplified.txt | 52 ++--- .../approved-plans-v1_4/q3/simplified.txt | 16 +- .../approved-plans-v1_4/q30/simplified.txt | 28 +-- .../approved-plans-v1_4/q31/simplified.txt | 68 +++---- .../approved-plans-v1_4/q32/simplified.txt | 20 +- .../approved-plans-v1_4/q33/simplified.txt | 50 ++--- .../approved-plans-v1_4/q34/simplified.txt | 26 +-- .../approved-plans-v1_4/q35/simplified.txt | 22 +-- .../approved-plans-v1_4/q36/simplified.txt | 24 +-- .../approved-plans-v1_4/q37/simplified.txt | 22 +-- .../approved-plans-v1_4/q38/simplified.txt | 28 +-- .../approved-plans-v1_4/q39a/simplified.txt | 38 ++-- .../approved-plans-v1_4/q39b/simplified.txt | 38 ++-- .../approved-plans-v1_4/q4/simplified.txt | 84 ++++----- .../approved-plans-v1_4/q40/simplified.txt | 10 +- .../approved-plans-v1_4/q41/simplified.txt | 6 +- .../approved-plans-v1_4/q42/simplified.txt | 16 +- .../approved-plans-v1_4/q43/simplified.txt | 16 +- .../approved-plans-v1_4/q44/simplified.txt | 10 +- .../approved-plans-v1_4/q45/simplified.txt | 28 +-- .../approved-plans-v1_4/q46/simplified.txt | 30 +-- .../approved-plans-v1_4/q47/simplified.txt | 22 +-- .../approved-plans-v1_4/q48/simplified.txt | 26 +-- .../approved-plans-v1_4/q49/simplified.txt | 42 ++--- .../approved-plans-v1_4/q5/simplified.txt | 58 +++--- .../approved-plans-v1_4/q50/simplified.txt | 28 +-- .../approved-plans-v1_4/q51/simplified.txt | 18 +- .../approved-plans-v1_4/q52/simplified.txt | 16 +- .../approved-plans-v1_4/q53/simplified.txt | 22 +-- .../approved-plans-v1_4/q54/simplified.txt | 32 ++-- .../approved-plans-v1_4/q55/simplified.txt | 16 +- .../approved-plans-v1_4/q56/simplified.txt | 50 ++--- .../approved-plans-v1_4/q57/simplified.txt | 22 +-- .../approved-plans-v1_4/q58/simplified.txt | 42 ++--- .../approved-plans-v1_4/q59/simplified.txt | 12 +- .../approved-plans-v1_4/q6/simplified.txt | 30 +-- .../approved-plans-v1_4/q60/simplified.txt | 50 ++--- .../approved-plans-v1_4/q61/simplified.txt | 54 +++--- .../approved-plans-v1_4/q62/simplified.txt | 28 +-- .../approved-plans-v1_4/q63/simplified.txt | 22 +-- .../approved-plans-v1_4/q64/simplified.txt | 32 ++-- .../approved-plans-v1_4/q65/simplified.txt | 22 +-- .../approved-plans-v1_4/q66/simplified.txt | 42 ++--- .../approved-plans-v1_4/q67/simplified.txt | 24 +-- .../approved-plans-v1_4/q68/simplified.txt | 30 +-- .../approved-plans-v1_4/q69/simplified.txt | 22 +-- .../approved-plans-v1_4/q7/simplified.txt | 30 +-- .../approved-plans-v1_4/q70/simplified.txt | 24 +-- .../approved-plans-v1_4/q71/simplified.txt | 34 ++-- .../approved-plans-v1_4/q72/simplified.txt | 50 ++--- .../approved-plans-v1_4/q73/simplified.txt | 26 +-- .../approved-plans-v1_4/q74/simplified.txt | 60 +++--- .../approved-plans-v1_4/q75/simplified.txt | 58 +++--- .../approved-plans-v1_4/q76/simplified.txt | 30 +-- .../approved-plans-v1_4/q77/simplified.txt | 50 ++--- .../approved-plans-v1_4/q78/simplified.txt | 14 +- .../approved-plans-v1_4/q79/simplified.txt | 26 +-- .../approved-plans-v1_4/q8/simplified.txt | 26 +-- .../approved-plans-v1_4/q80/simplified.txt | 24 +-- .../approved-plans-v1_4/q81/simplified.txt | 28 +-- .../approved-plans-v1_4/q82/simplified.txt | 22 +-- .../approved-plans-v1_4/q83/simplified.txt | 44 ++--- .../approved-plans-v1_4/q84/simplified.txt | 28 +-- .../approved-plans-v1_4/q85/simplified.txt | 42 ++--- .../approved-plans-v1_4/q86/simplified.txt | 18 +- .../approved-plans-v1_4/q87/simplified.txt | 28 +-- .../approved-plans-v1_4/q88/simplified.txt | 104 +++++----- .../approved-plans-v1_4/q89/simplified.txt | 24 +-- .../approved-plans-v1_4/q9/simplified.txt | 20 +- .../approved-plans-v1_4/q90/simplified.txt | 32 ++-- .../approved-plans-v1_4/q91/simplified.txt | 42 ++--- .../approved-plans-v1_4/q92/simplified.txt | 20 +- .../approved-plans-v1_4/q93/simplified.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 8 +- .../approved-plans-v1_4/q95/simplified.txt | 12 +- .../approved-plans-v1_4/q96/simplified.txt | 20 +- .../approved-plans-v1_4/q97/simplified.txt | 10 +- .../approved-plans-v1_4/q98/simplified.txt | 18 +- .../approved-plans-v1_4/q99/simplified.txt | 28 +-- .../approved-plans-v2_7/q10a/simplified.txt | 38 ++-- .../approved-plans-v2_7/q11/simplified.txt | 60 +++--- .../approved-plans-v2_7/q12/simplified.txt | 18 +- .../approved-plans-v2_7/q14/simplified.txt | 58 +++--- .../approved-plans-v2_7/q14a/simplified.txt | 62 +++--- .../approved-plans-v2_7/q18a/simplified.txt | 118 ++++++------ .../approved-plans-v2_7/q20/simplified.txt | 18 +- .../approved-plans-v2_7/q22/simplified.txt | 16 +- .../approved-plans-v2_7/q22a/simplified.txt | 22 +-- .../approved-plans-v2_7/q24/simplified.txt | 14 +- .../approved-plans-v2_7/q27a/simplified.txt | 60 +++--- .../approved-plans-v2_7/q34/simplified.txt | 26 +-- .../approved-plans-v2_7/q35/simplified.txt | 22 +-- .../approved-plans-v2_7/q35a/simplified.txt | 38 ++-- .../approved-plans-v2_7/q36a/simplified.txt | 24 +-- .../approved-plans-v2_7/q47/simplified.txt | 22 +-- .../approved-plans-v2_7/q49/simplified.txt | 42 ++--- .../approved-plans-v2_7/q51a/simplified.txt | 18 +- .../approved-plans-v2_7/q57/simplified.txt | 22 +-- .../approved-plans-v2_7/q5a/simplified.txt | 58 +++--- .../approved-plans-v2_7/q6/simplified.txt | 30 +-- .../approved-plans-v2_7/q64/simplified.txt | 32 ++-- .../approved-plans-v2_7/q67a/simplified.txt | 24 +-- .../approved-plans-v2_7/q70a/simplified.txt | 24 +-- .../approved-plans-v2_7/q72/simplified.txt | 50 ++--- .../approved-plans-v2_7/q74/simplified.txt | 60 +++--- .../approved-plans-v2_7/q75/simplified.txt | 58 +++--- .../approved-plans-v2_7/q77a/simplified.txt | 50 ++--- .../approved-plans-v2_7/q78/simplified.txt | 14 +- .../approved-plans-v2_7/q80a/simplified.txt | 24 +-- .../approved-plans-v2_7/q86a/simplified.txt | 18 +- .../approved-plans-v2_7/q98/simplified.txt | 18 +- 145 files changed, 2312 insertions(+), 2265 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index fb1f62542..e0a23e41e 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -328,13 +328,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometProjectExec( + val newPlan = CometProjectExec( nativeOp, - op, - op.projectList, op.output, + op.projectList, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -343,7 +343,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometFilterExec(nativeOp, op, op.condition, op.child, SerializedPlan(None)) + val newPlan = + CometFilterExec(nativeOp, op.output, op.condition, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -352,7 +354,15 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometSortExec(nativeOp, op, op.sortOrder, op.child, SerializedPlan(None)) + val newPlan = + CometSortExec( + nativeOp, + op.output, + op.outputOrdering, + op.sortOrder, + op.child, + SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -361,7 +371,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometLocalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) + val newPlan = + CometLocalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -370,7 +382,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometGlobalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) + val newPlan = + CometGlobalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -382,8 +396,9 @@ class CometSparkSessionExtensions QueryPlanSerde.operator2Proto(op) match { case Some(nativeOp) => val offset = getOffset(op) - val cometOp = - CometCollectLimitExec(op, op.limit, offset, op.child) + val newPlan = + CometCollectLimitExec(op.limit, offset, op.child) + val cometOp = setLogicalLink(newPlan, op) CometSinkPlaceHolder(nativeOp, op, cometOp) case None => op @@ -393,12 +408,28 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometExpandExec(nativeOp, op, op.projections, op.child, SerializedPlan(None)) + val newPlan = + CometExpandExec( + nativeOp, + op.output, + op.projections, + op.child, + SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } - case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) => + case op @ HashAggregateExec( + _, + _, + _, + groupingExprs, + aggExprs, + _, + _, + resultExpressions, + child) => val modes = aggExprs.map(_.mode).distinct if (!modes.isEmpty && modes.size != 1) { @@ -422,15 +453,17 @@ class CometSparkSessionExtensions // modes is empty too. If aggExprs is not empty, we need to verify all the // aggregates have the same mode. assert(modes.length == 1 || modes.length == 0) - CometHashAggregateExec( + val newPlan = CometHashAggregateExec( nativeOp, - op, + op.output, groupingExprs, aggExprs, + resultExpressions, child.output, if (modes.nonEmpty) Some(modes.head) else None, child, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -443,9 +476,10 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometHashJoinExec( + val newPlan = CometHashJoinExec( nativeOp, - op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -454,6 +488,7 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -475,9 +510,10 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometBroadcastHashJoinExec( + val newPlan = CometBroadcastHashJoinExec( nativeOp, - op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -486,6 +522,7 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -496,9 +533,10 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - CometSortMergeJoinExec( + val newPlan = CometSortMergeJoinExec( nativeOp, - op, + op.output, + op.outputOrdering, op.leftKeys, op.rightKeys, op.joinType, @@ -506,6 +544,7 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) + setLogicalLink(newPlan, op) case None => op } @@ -535,7 +574,8 @@ class CometSparkSessionExtensions && isCometNative(child) => QueryPlanSerde.operator2Proto(c) match { case Some(nativeOp) => - val cometOp = CometCoalesceExec(c, numPartitions, child) + val newPlan = CometCoalesceExec(c.output, numPartitions, child) + val cometOp = setLogicalLink(newPlan, c) CometSinkPlaceHolder(nativeOp, c, cometOp) case None => c @@ -558,8 +598,14 @@ class CometSparkSessionExtensions CometTakeOrderedAndProjectExec.isSupported(s) => QueryPlanSerde.operator2Proto(s) match { case Some(nativeOp) => - val cometOp = - CometTakeOrderedAndProjectExec(s, s.limit, s.sortOrder, s.projectList, s.child) + val newPlan = + CometTakeOrderedAndProjectExec( + s.output, + s.limit, + s.sortOrder, + s.projectList, + s.child) + val cometOp = setLogicalLink(newPlan, s) CometSinkPlaceHolder(nativeOp, s, cometOp) case None => s @@ -579,8 +625,14 @@ class CometSparkSessionExtensions val newOp = transform1(w) newOp match { case Some(nativeOp) => - val cometOp = - CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) + val newPlan = + CometWindowExec( + w.output, + w.windowExpression, + w.partitionSpec, + w.orderSpec, + w.child) + val cometOp = setLogicalLink(newPlan, w) CometSinkPlaceHolder(nativeOp, w, cometOp) case None => w @@ -591,7 +643,8 @@ class CometSparkSessionExtensions u.children.forall(isCometNative) => QueryPlanSerde.operator2Proto(u) match { case Some(nativeOp) => - val cometOp = CometUnionExec(u, u.children) + val newPlan = CometUnionExec(u.output, u.children) + val cometOp = setLogicalLink(newPlan, u) CometSinkPlaceHolder(nativeOp, u, cometOp) case None => u @@ -631,7 +684,8 @@ class CometSparkSessionExtensions isSpark34Plus => // Spark 3.4+ only QueryPlanSerde.operator2Proto(b) match { case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.child) + val newPlan = CometBroadcastExchangeExec(b.output, b.child) + val cometOp = setLogicalLink(newPlan, b) CometSinkPlaceHolder(nativeOp, b, cometOp) case None => b } @@ -822,40 +876,6 @@ class CometSparkSessionExtensions case CometScanWrapper(_, s) => s } - // Set up logical links - newPlan = newPlan.transform { - case op: CometExec => - if (op.originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - op.originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - case op: CometShuffleExchangeExec => - // Original Spark shuffle exchange operator might have empty logical link. - // But the `setLogicalLink` call above on downstream operator of - // `CometShuffleExchangeExec` will set its logical link to the downstream - // operators which cause AQE behavior to be incorrect. So we need to unset - // the logical link here. - if (op.originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - op.originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - - case op: CometBroadcastExchangeExec => - if (op.originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - op.originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - } - // Convert native execution block by linking consecutive native operators. var firstNativeOp = true newPlan.transformDown { @@ -887,6 +907,44 @@ class CometSparkSessionExtensions }.flatten } + /** + * Set up logical links for transformed Comet operators. + */ + def setLogicalLink(newPlan: SparkPlan, originalPlan: SparkPlan): SparkPlan = { + newPlan match { + case op: CometExec => + if (originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + case op: CometShuffleExchangeExec => + // Original Spark shuffle exchange operator might have empty logical link. + // But the `setLogicalLink` call above on downstream operator of + // `CometShuffleExchangeExec` will set its logical link to the downstream + // operators which cause AQE behavior to be incorrect. So we need to unset + // the logical link here. + if (originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + + case op: CometBroadcastExchangeExec => + if (originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + } + } + /** * Returns true if a given spark plan is Comet shuffle operator. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index 38247b2c4..426806916 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -31,6 +31,7 @@ import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} @@ -60,7 +61,9 @@ import org.apache.comet.CometRuntimeException * Note that this only supports Spark 3.4 and later, because the serialization class * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later. */ -case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) +case class CometBroadcastExchangeExec( + override val output: Seq[Attribute], + override val child: SparkPlan) extends BroadcastExchangeLike with ShimCometBroadcastExchangeExec { import CometBroadcastExchangeExec._ @@ -74,10 +77,6 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) - override def doCanonicalize(): SparkPlan = { - CometBroadcastExchangeExec(originalPlan.canonicalized, child.canonicalized) - } - override def runtimeStatistics: Statistics = { val dataSize = metrics("dataSize").value val rowCount = metrics("numOutputRows").value @@ -237,7 +236,7 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan) override def equals(obj: Any): Boolean = { obj match { case other: CometBroadcastExchangeExec => - this.originalPlan == other.originalPlan && + this.output == other.output && this.child == other.child case _ => false diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala index cc635d739..1f656b117 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -31,7 +32,7 @@ import com.google.common.base.Objects * more efficient when including it in a Comet query plan. */ case class CometCoalesceExec( - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], numPartitions: Int, child: SparkPlan) extends CometExec diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index d2c9158ee..0090c537c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -22,6 +22,8 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME} import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer} @@ -38,13 +40,11 @@ import com.google.common.base.Objects * * TODO: support offset semantics */ -case class CometCollectLimitExec( - override val originalPlan: SparkPlan, - limit: Int, - offset: Int, - child: SparkPlan) +case class CometCollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends CometExec with UnaryExecNode { + override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = SinglePartition private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 6e9bfe424..2f5121bed 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -41,15 +41,13 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec * contains two native executions separated by a Comet shuffle exchange. */ case class CometTakeOrderedAndProjectExec( - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], limit: Int, sortOrder: Seq[SortOrder], projectList: Seq[NamedExpression], child: SparkPlan) extends CometExec with UnaryExecNode { - override def output: Seq[Attribute] = projectList.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 9685e75e1..221b725d5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -42,7 +42,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi * executions separated by a Comet shuffle exchange. */ case class CometWindowExec( - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], @@ -52,8 +52,6 @@ case class CometWindowExec( override def nodeName: String = "CometWindowExec" - override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) - private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) private lazy val readMetrics = diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index cec64d7a8..5d4848216 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -64,7 +64,6 @@ import org.apache.comet.shims.ShimCometShuffleExchangeExec case class CometShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, - originalPlan: ShuffleExchangeLike, shuffleOrigin: ShuffleOrigin = ENSURE_REQUIREMENTS, shuffleType: ShuffleType = CometNativeShuffle, advisoryPartitionSize: Option[Long] = None) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index ca3f69cdd..79989bd70 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf @@ -57,28 +56,15 @@ import org.apache.comet.shims.ShimCometBroadcastHashJoinExec */ abstract class CometExec extends CometPlan { - /** The original Spark operator from which this Comet operator is converted from */ - def originalPlan: SparkPlan - /** Comet always support columnar execution */ override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = originalPlan.output - override def doExecute(): RDD[InternalRow] = ColumnarToRowExec(this).doExecute() override def executeCollect(): Array[InternalRow] = ColumnarToRowExec(this).executeCollect() - override def outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering - - // `CometExec` reuses the outputPartitioning of the original SparkPlan. - // Note that if the outputPartitioning of the original SparkPlan depends on its children, - // we should override this method in the specific CometExec, because Spark AQE may change the - // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. - override def outputPartitioning: Partitioning = originalPlan.outputPartitioning - /** * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. */ @@ -154,7 +140,7 @@ abstract class CometNativeExec extends CometExec { /** The Comet native operator */ def nativeOp: Operator - override protected def doPrepare(): Unit = prepareSubqueries(originalPlan) + override protected def doPrepare(): Unit = prepareSubqueries(this) override lazy val metrics: Map[String, SQLMetric] = CometMetricNode.baselineMetrics(sparkContext) @@ -217,12 +203,12 @@ abstract class CometNativeExec extends CometExec { val it = new CometExecIterator(CometExec.newIterId, inputs, serializedPlanCopy, nativeMetrics) - setSubqueries(it.id, originalPlan) + setSubqueries(it.id, this) Option(TaskContext.get()).foreach { context => context.addTaskCompletionListener[Unit] { _ => it.close() - cleanSubqueries(it.id, originalPlan) + cleanSubqueries(it.id, this) } } @@ -363,8 +349,7 @@ abstract class CometNativeExec extends CometExec { } override protected def doCanonicalize(): SparkPlan = { - val canonicalizedPlan = super - .doCanonicalize() + val canonicalizedPlan = super.doCanonicalize() .asInstanceOf[CometNativeExec] .canonicalizePlans() @@ -413,9 +398,8 @@ case class SerializedPlan(plan: Option[Array[Byte]]) { case class CometProjectExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, - projectList: Seq[NamedExpression], override val output: Seq[Attribute], + projectList: Seq[NamedExpression], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec @@ -444,19 +428,20 @@ case class CometProjectExec( case class CometFilterExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], condition: Expression, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, condition, child) + Iterator(output, condition, child) override def equals(obj: Any): Boolean = { obj match { @@ -481,7 +466,8 @@ case class CometFilterExec( case class CometSortExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], sortOrder: Seq[SortOrder], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -493,7 +479,7 @@ case class CometSortExec( this.copy(child = newChild) override def stringArgs: Iterator[Any] = - Iterator(originalPlan.output, sortOrder, child) + Iterator(output, sortOrder, child) override def equals(obj: Any): Boolean = { obj match { @@ -516,13 +502,14 @@ case class CometSortExec( case class CometLocalLimitExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -546,13 +533,14 @@ case class CometLocalLimitExec( case class CometGlobalLimitExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = this.copy(child = newChild) @@ -574,11 +562,13 @@ case class CometGlobalLimitExec( case class CometExpandExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], projections: Seq[Seq[Expression]], child: SparkPlan, override val serializedPlanOpt: SerializedPlan) extends CometUnaryExec { + override def outputPartitioning: Partitioning = UnknownPartitioning(0) + override def producedAttributes: AttributeSet = outputSet override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -602,7 +592,7 @@ case class CometExpandExec( override lazy val metrics: Map[String, SQLMetric] = Map.empty } -case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan]) +case class CometUnionExec(override val output: Seq[Attribute], children: Seq[SparkPlan]) extends CometExec { override def doExecuteColumnar(): RDD[ColumnarBatch] = { sparkContext.union(children.map(_.executeColumnar())) @@ -635,9 +625,10 @@ case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[Sp case class CometHashAggregateExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression], input: Seq[Attribute], mode: Option[AggregateMode], child: SparkPlan, @@ -676,13 +667,13 @@ case class CometHashAggregateExec( override def hashCode(): Int = Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child) - override protected def outputExpressions: Seq[NamedExpression] = - originalPlan.asInstanceOf[HashAggregateExec].resultExpressions + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions } case class CometHashJoinExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -734,7 +725,8 @@ case class CometHashJoinExec( case class CometBroadcastHashJoinExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -868,7 +860,8 @@ case class CometBroadcastHashJoinExec( case class CometSortMergeJoinExec( override val nativeOp: Operator, - override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, @@ -923,9 +916,10 @@ case class CometSortMergeJoinExec( "join_time" -> SQLMetrics.createNanoTimingMetric(sparkContext, "Total time for joining")) } -case class CometScanWrapper(override val nativeOp: Operator, override val originalPlan: SparkPlan) +case class CometScanWrapper(override val nativeOp: Operator, originalPlan: SparkPlan) extends CometNativeExec with LeafExecNode { + override def output: Seq[Attribute] = originalPlan.output override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override def stringArgs: Iterator[Any] = Iterator(originalPlan.output, originalPlan) } @@ -939,9 +933,11 @@ case class CometScanWrapper(override val nativeOp: Operator, override val origin */ case class CometSinkPlaceHolder( override val nativeOp: Operator, // Must be a Scan - override val originalPlan: SparkPlan, + originalPlan: SparkPlan, child: SparkPlan) extends CometUnaryExec { + override def output: Seq[Attribute] = originalPlan.output + override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 965b6851e..9fadcc680 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -35,7 +35,6 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, - s, s.shuffleOrigin, shuffleType, advisoryPartitionSize) diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 559e327b4..2be852ae5 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -31,7 +31,6 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, - s, s.shuffleOrigin, shuffleType, s.advisoryPartitionSize) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 000160518..238f9b702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [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,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index 50472502e..60cad82dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 701a90912..f713176f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -27,9 +27,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -37,9 +37,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -57,7 +57,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index c4ea8fe24..537cba446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index b529cb5ff..c37569802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,15 +30,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index f4a03a2d2..f95e8d040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -57,15 +57,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index f80554fb5..e576a079b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index ecae29c78..2313467ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index dd66c9582..1428c7ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index a3a99e312..dfa9ae66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,17 +30,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 9e72edbb4..feca96b18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -64,16 +64,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] From ed291d975b1a1bacc06b223d4a5f9bf3e0f8fd6f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 11:05:49 -0700 Subject: [PATCH 5/9] More --- .../src/main/scala/org/apache/spark/sql/comet/operators.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 79989bd70..5e3f08f23 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -349,7 +349,8 @@ abstract class CometNativeExec extends CometExec { } override protected def doCanonicalize(): SparkPlan = { - val canonicalizedPlan = super.doCanonicalize() + val canonicalizedPlan = super + .doCanonicalize() .asInstanceOf[CometNativeExec] .canonicalizePlans() From 4654346cc6d8c0c24c817d638005b9af4389b036 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 12:58:26 -0700 Subject: [PATCH 6/9] More --- .../comet/CometSparkSessionExtensions.scala | 164 ++++++++---------- .../comet/CometBroadcastExchangeExec.scala | 7 +- .../spark/sql/comet/CometCoalesceExec.scala | 1 + .../sql/comet/CometCollectLimitExec.scala | 6 +- .../CometTakeOrderedAndProjectExec.scala | 1 + .../spark/sql/comet/CometWindowExec.scala | 1 + .../shuffle/CometShuffleExchangeExec.scala | 1 + .../apache/spark/sql/comet/operators.scala | 35 +++- .../shims/ShimCometShuffleExchangeExec.scala | 1 + 9 files changed, 122 insertions(+), 95 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index e0a23e41e..c30d6b0f7 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -328,13 +328,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometProjectExec( + CometProjectExec( nativeOp, + op, op.output, op.projectList, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -343,9 +343,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometFilterExec(nativeOp, op.output, op.condition, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometFilterExec( + nativeOp, + op, + op.output, + op.condition, + op.child, + SerializedPlan(None)) case None => op } @@ -354,15 +358,14 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometSortExec( - nativeOp, - op.output, - op.outputOrdering, - op.sortOrder, - op.child, - SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometSortExec( + nativeOp, + op, + op.output, + op.outputOrdering, + op.sortOrder, + op.child, + SerializedPlan(None)) case None => op } @@ -371,9 +374,7 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometLocalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometLocalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) case None => op } @@ -382,9 +383,7 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometGlobalLimitExec(nativeOp, op.limit, op.child, SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometGlobalLimitExec(nativeOp, op, op.limit, op.child, SerializedPlan(None)) case None => op } @@ -396,9 +395,8 @@ class CometSparkSessionExtensions QueryPlanSerde.operator2Proto(op) match { case Some(nativeOp) => val offset = getOffset(op) - val newPlan = - CometCollectLimitExec(op.limit, offset, op.child) - val cometOp = setLogicalLink(newPlan, op) + val cometOp = + CometCollectLimitExec(op, op.limit, offset, op.child) CometSinkPlaceHolder(nativeOp, op, cometOp) case None => op @@ -408,14 +406,13 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = - CometExpandExec( - nativeOp, - op.output, - op.projections, - op.child, - SerializedPlan(None)) - setLogicalLink(newPlan, op) + CometExpandExec( + nativeOp, + op, + op.output, + op.projections, + op.child, + SerializedPlan(None)) case None => op } @@ -453,8 +450,9 @@ class CometSparkSessionExtensions // modes is empty too. If aggExprs is not empty, we need to verify all the // aggregates have the same mode. assert(modes.length == 1 || modes.length == 0) - val newPlan = CometHashAggregateExec( + CometHashAggregateExec( nativeOp, + op, op.output, groupingExprs, aggExprs, @@ -463,7 +461,6 @@ class CometSparkSessionExtensions if (modes.nonEmpty) Some(modes.head) else None, child, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -476,8 +473,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometHashJoinExec( + CometHashJoinExec( nativeOp, + op, op.output, op.outputOrdering, op.leftKeys, @@ -488,7 +486,6 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -510,8 +507,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometBroadcastHashJoinExec( + CometBroadcastHashJoinExec( nativeOp, + op, op.output, op.outputOrdering, op.leftKeys, @@ -522,7 +520,6 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -533,8 +530,9 @@ class CometSparkSessionExtensions val newOp = transform1(op) newOp match { case Some(nativeOp) => - val newPlan = CometSortMergeJoinExec( + CometSortMergeJoinExec( nativeOp, + op, op.output, op.outputOrdering, op.leftKeys, @@ -544,7 +542,6 @@ class CometSparkSessionExtensions op.left, op.right, SerializedPlan(None)) - setLogicalLink(newPlan, op) case None => op } @@ -574,8 +571,7 @@ class CometSparkSessionExtensions && isCometNative(child) => QueryPlanSerde.operator2Proto(c) match { case Some(nativeOp) => - val newPlan = CometCoalesceExec(c.output, numPartitions, child) - val cometOp = setLogicalLink(newPlan, c) + val cometOp = CometCoalesceExec(c, c.output, numPartitions, child) CometSinkPlaceHolder(nativeOp, c, cometOp) case None => c @@ -598,14 +594,14 @@ class CometSparkSessionExtensions CometTakeOrderedAndProjectExec.isSupported(s) => QueryPlanSerde.operator2Proto(s) match { case Some(nativeOp) => - val newPlan = + val cometOp = CometTakeOrderedAndProjectExec( + s, s.output, s.limit, s.sortOrder, s.projectList, s.child) - val cometOp = setLogicalLink(newPlan, s) CometSinkPlaceHolder(nativeOp, s, cometOp) case None => s @@ -625,14 +621,14 @@ class CometSparkSessionExtensions val newOp = transform1(w) newOp match { case Some(nativeOp) => - val newPlan = + val cometOp = CometWindowExec( + w, w.output, w.windowExpression, w.partitionSpec, w.orderSpec, w.child) - val cometOp = setLogicalLink(newPlan, w) CometSinkPlaceHolder(nativeOp, w, cometOp) case None => w @@ -643,8 +639,7 @@ class CometSparkSessionExtensions u.children.forall(isCometNative) => QueryPlanSerde.operator2Proto(u) match { case Some(nativeOp) => - val newPlan = CometUnionExec(u.output, u.children) - val cometOp = setLogicalLink(newPlan, u) + val cometOp = CometUnionExec(u, u.output, u.children) CometSinkPlaceHolder(nativeOp, u, cometOp) case None => u @@ -684,8 +679,7 @@ class CometSparkSessionExtensions isSpark34Plus => // Spark 3.4+ only QueryPlanSerde.operator2Proto(b) match { case Some(nativeOp) => - val newPlan = CometBroadcastExchangeExec(b.output, b.child) - val cometOp = setLogicalLink(newPlan, b) + val cometOp = CometBroadcastExchangeExec(b, b.output, b.child) CometSinkPlaceHolder(nativeOp, b, cometOp) case None => b } @@ -876,6 +870,40 @@ class CometSparkSessionExtensions case CometScanWrapper(_, s) => s } + // Set up logical links + newPlan = newPlan.transform { + case op: CometExec => + if (op.originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + case op: CometShuffleExchangeExec => + // Original Spark shuffle exchange operator might have empty logical link. + // But the `setLogicalLink` call above on downstream operator of + // `CometShuffleExchangeExec` will set its logical link to the downstream + // operators which cause AQE behavior to be incorrect. So we need to unset + // the logical link here. + if (op.originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + + case op: CometBroadcastExchangeExec => + if (op.originalPlan.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + } + op + } + // Convert native execution block by linking consecutive native operators. var firstNativeOp = true newPlan.transformDown { @@ -907,44 +935,6 @@ class CometSparkSessionExtensions }.flatten } - /** - * Set up logical links for transformed Comet operators. - */ - def setLogicalLink(newPlan: SparkPlan, originalPlan: SparkPlan): SparkPlan = { - newPlan match { - case op: CometExec => - if (originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - case op: CometShuffleExchangeExec => - // Original Spark shuffle exchange operator might have empty logical link. - // But the `setLogicalLink` call above on downstream operator of - // `CometShuffleExchangeExec` will set its logical link to the downstream - // operators which cause AQE behavior to be incorrect. So we need to unset - // the logical link here. - if (originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - - case op: CometBroadcastExchangeExec => - if (originalPlan.logicalLink.isEmpty) { - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) - op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) - } else { - originalPlan.logicalLink.foreach(op.setLogicalLink) - } - op - } - } - /** * Returns true if a given spark plan is Comet shuffle operator. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index 426806916..64d14a00e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -62,6 +62,7 @@ import org.apache.comet.CometRuntimeException * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later. */ case class CometBroadcastExchangeExec( + originalPlan: SparkPlan, override val output: Seq[Attribute], override val child: SparkPlan) extends BroadcastExchangeLike @@ -77,6 +78,10 @@ case class CometBroadcastExchangeExec( "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) + override def doCanonicalize(): SparkPlan = { + CometBroadcastExchangeExec(null, null, child.canonicalized) + } + override def runtimeStatistics: Statistics = { val dataSize = metrics("dataSize").value val rowCount = metrics("numOutputRows").value @@ -236,7 +241,7 @@ case class CometBroadcastExchangeExec( override def equals(obj: Any): Boolean = { obj match { case other: CometBroadcastExchangeExec => - this.output == other.output && + this.originalPlan == other.originalPlan && this.child == other.child case _ => false diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala index 1f656b117..b216c7d81 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -32,6 +32,7 @@ import com.google.common.base.Objects * more efficient when including it in a Comet query plan. */ case class CometCoalesceExec( + override val originalPlan: SparkPlan, override val output: Seq[Attribute], numPartitions: Int, child: SparkPlan) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala index 0090c537c..3effcca4f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala @@ -40,7 +40,11 @@ import com.google.common.base.Objects * * TODO: support offset semantics */ -case class CometCollectLimitExec(limit: Int, offset: Int, child: SparkPlan) +case class CometCollectLimitExec( + override val originalPlan: SparkPlan, + limit: Int, + offset: Int, + child: SparkPlan) extends CometExec with UnaryExecNode { override def output: Seq[Attribute] = child.output diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala index 2f5121bed..3f09e2fe2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala @@ -41,6 +41,7 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec * contains two native executions separated by a Comet shuffle exchange. */ case class CometTakeOrderedAndProjectExec( + override val originalPlan: SparkPlan, override val output: Seq[Attribute], limit: Int, sortOrder: Seq[SortOrder], diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala index 221b725d5..4c10a8abb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala @@ -42,6 +42,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi * executions separated by a Comet shuffle exchange. */ case class CometWindowExec( + override val originalPlan: SparkPlan, override val output: Seq[Attribute], windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 5d4848216..cec64d7a8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -64,6 +64,7 @@ import org.apache.comet.shims.ShimCometShuffleExchangeExec case class CometShuffleExchangeExec( override val outputPartitioning: Partitioning, child: SparkPlan, + originalPlan: ShuffleExchangeLike, shuffleOrigin: ShuffleOrigin = ENSURE_REQUIREMENTS, shuffleType: ShuffleType = CometNativeShuffle, advisoryPartitionSize: Option[Long] = None) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 5e3f08f23..1c9bf9947 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -56,15 +56,28 @@ import org.apache.comet.shims.ShimCometBroadcastHashJoinExec */ abstract class CometExec extends CometPlan { + /** The original Spark operator from which this Comet operator is converted from */ + def originalPlan: SparkPlan + /** Comet always support columnar execution */ override def supportsColumnar: Boolean = true + override def output: Seq[Attribute] = originalPlan.output + override def doExecute(): RDD[InternalRow] = ColumnarToRowExec(this).doExecute() override def executeCollect(): Array[InternalRow] = ColumnarToRowExec(this).executeCollect() + override def outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering + + // `CometExec` reuses the outputPartitioning of the original SparkPlan. + // Note that if the outputPartitioning of the original SparkPlan depends on its children, + // we should override this method in the specific CometExec, because Spark AQE may change the + // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. + override def outputPartitioning: Partitioning = originalPlan.outputPartitioning + /** * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. */ @@ -399,6 +412,7 @@ case class SerializedPlan(plan: Option[Array[Byte]]) { case class CometProjectExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], projectList: Seq[NamedExpression], child: SparkPlan, @@ -429,6 +443,7 @@ case class CometProjectExec( case class CometFilterExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], condition: Expression, child: SparkPlan, @@ -467,6 +482,7 @@ case class CometFilterExec( case class CometSortExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], override val outputOrdering: Seq[SortOrder], sortOrder: Seq[SortOrder], @@ -503,6 +519,7 @@ case class CometSortExec( case class CometLocalLimitExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -534,6 +551,7 @@ case class CometLocalLimitExec( case class CometGlobalLimitExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, limit: Int, child: SparkPlan, override val serializedPlanOpt: SerializedPlan) @@ -563,6 +581,7 @@ case class CometGlobalLimitExec( case class CometExpandExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], projections: Seq[Seq[Expression]], child: SparkPlan, @@ -593,7 +612,10 @@ case class CometExpandExec( override lazy val metrics: Map[String, SQLMetric] = Map.empty } -case class CometUnionExec(override val output: Seq[Attribute], children: Seq[SparkPlan]) +case class CometUnionExec( + override val originalPlan: SparkPlan, + override val output: Seq[Attribute], + children: Seq[SparkPlan]) extends CometExec { override def doExecuteColumnar(): RDD[ColumnarBatch] = { sparkContext.union(children.map(_.executeColumnar())) @@ -626,6 +648,7 @@ case class CometUnionExec(override val output: Seq[Attribute], children: Seq[Spa case class CometHashAggregateExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], @@ -673,6 +696,7 @@ case class CometHashAggregateExec( case class CometHashJoinExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], @@ -726,6 +750,7 @@ case class CometHashJoinExec( case class CometBroadcastHashJoinExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], @@ -861,6 +886,7 @@ case class CometBroadcastHashJoinExec( case class CometSortMergeJoinExec( override val nativeOp: Operator, + override val originalPlan: SparkPlan, override val output: Seq[Attribute], override val outputOrdering: Seq[SortOrder], leftKeys: Seq[Expression], @@ -917,10 +943,9 @@ case class CometSortMergeJoinExec( "join_time" -> SQLMetrics.createNanoTimingMetric(sparkContext, "Total time for joining")) } -case class CometScanWrapper(override val nativeOp: Operator, originalPlan: SparkPlan) +case class CometScanWrapper(override val nativeOp: Operator, override val originalPlan: SparkPlan) extends CometNativeExec with LeafExecNode { - override def output: Seq[Attribute] = originalPlan.output override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override def stringArgs: Iterator[Any] = Iterator(originalPlan.output, originalPlan) } @@ -934,11 +959,9 @@ case class CometScanWrapper(override val nativeOp: Operator, originalPlan: Spark */ case class CometSinkPlaceHolder( override val nativeOp: Operator, // Must be a Scan - originalPlan: SparkPlan, + override val originalPlan: SparkPlan, child: SparkPlan) extends CometUnaryExec { - override def output: Seq[Attribute] = originalPlan.output - override val serializedPlanOpt: SerializedPlan = SerializedPlan(None) override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 9fadcc680..965b6851e 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -35,6 +35,7 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, + s, s.shuffleOrigin, shuffleType, advisoryPartitionSize) From e84ba5edf0a79a25893b89eaf4a6f24e3ed16b63 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 13:09:51 -0700 Subject: [PATCH 7/9] Fix --- .../org/apache/comet/shims/ShimCometShuffleExchangeExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 2be852ae5..559e327b4 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -31,6 +31,7 @@ trait ShimCometShuffleExchangeExec { CometShuffleExchangeExec( s.outputPartitioning, s.child, + s, s.shuffleOrigin, shuffleType, s.advisoryPartitionSize) From 2e1998656833016386a18d07ba47e1544c222401 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 14:04:40 -0700 Subject: [PATCH 8/9] Fix diffs --- dev/diffs/3.4.3.diff | 4 ++-- dev/diffs/3.5.1.diff | 4 ++-- dev/diffs/4.0.0-preview1.diff | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8129c7455..3c88d6a7e 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2491,8 +2491,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff index ebc4cdb8c..cf6754416 100644 --- a/dev/diffs/3.5.1.diff +++ b/dev/diffs/3.5.1.diff @@ -2650,8 +2650,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 4031015df..744722ecf 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -2651,8 +2651,8 @@ index 5fbf379644f..32711763ec1 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) From 35071b139bd4294edcc5df5d39bbe7727a7d2791 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jul 2024 15:27:42 -0700 Subject: [PATCH 9/9] Update --- .../q1/simplified.txt | 22 ++-- .../q10/simplified.txt | 22 ++-- .../q11/simplified.txt | 60 ++++----- .../q12/simplified.txt | 18 +-- .../q13/simplified.txt | 30 ++--- .../q14a/simplified.txt | 56 ++++----- .../q14b/simplified.txt | 58 ++++----- .../q15/simplified.txt | 22 ++-- .../q16/simplified.txt | 8 +- .../q17/simplified.txt | 46 +++---- .../q18/simplified.txt | 40 +++--- .../q19/simplified.txt | 30 ++--- .../q2/simplified.txt | 14 +-- .../q20/simplified.txt | 18 +-- .../q21/simplified.txt | 24 ++-- .../q22/simplified.txt | 22 ++-- .../q23a/simplified.txt | 42 +++---- .../q23b/simplified.txt | 48 +++---- .../q24a/simplified.txt | 14 +-- .../q24b/simplified.txt | 14 +-- .../q25/simplified.txt | 46 +++---- .../q26/simplified.txt | 30 ++--- .../q27/simplified.txt | 30 ++--- .../q28/simplified.txt | 24 ++-- .../q29/simplified.txt | 52 ++++---- .../q3/simplified.txt | 16 +-- .../q30/simplified.txt | 28 ++--- .../q31/simplified.txt | 68 +++++----- .../q32/simplified.txt | 20 +-- .../q33/simplified.txt | 50 ++++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q36/simplified.txt | 24 ++-- .../q37/simplified.txt | 22 ++-- .../q38/simplified.txt | 28 ++--- .../q39a/simplified.txt | 38 +++--- .../q39b/simplified.txt | 38 +++--- .../q4/simplified.txt | 84 ++++++------- .../q40/simplified.txt | 10 +- .../q41/simplified.txt | 6 +- .../q42/simplified.txt | 16 +-- .../q43/simplified.txt | 16 +-- .../q44/simplified.txt | 10 +- .../q45/simplified.txt | 28 ++--- .../q46/simplified.txt | 30 ++--- .../q47/simplified.txt | 22 ++-- .../q48/simplified.txt | 26 ++-- .../q49/simplified.txt | 42 +++---- .../q5/simplified.txt | 58 ++++----- .../q50/simplified.txt | 28 ++--- .../q51/simplified.txt | 18 +-- .../q52/simplified.txt | 16 +-- .../q53/simplified.txt | 22 ++-- .../q54/simplified.txt | 32 ++--- .../q55/simplified.txt | 16 +-- .../q56/simplified.txt | 50 ++++---- .../q57/simplified.txt | 22 ++-- .../q58/simplified.txt | 42 +++---- .../q59/simplified.txt | 12 +- .../q6/simplified.txt | 30 ++--- .../q60/simplified.txt | 50 ++++---- .../q61/simplified.txt | 54 ++++---- .../q62/simplified.txt | 28 ++--- .../q63/simplified.txt | 22 ++-- .../q64/simplified.txt | 32 ++--- .../q65/simplified.txt | 22 ++-- .../q66/simplified.txt | 42 +++---- .../q67/simplified.txt | 24 ++-- .../q68/simplified.txt | 30 ++--- .../q69/simplified.txt | 22 ++-- .../q7/simplified.txt | 30 ++--- .../q70/simplified.txt | 24 ++-- .../q71/simplified.txt | 34 ++--- .../q72/simplified.txt | 50 ++++---- .../q73/simplified.txt | 26 ++-- .../q74/simplified.txt | 60 ++++----- .../q75/simplified.txt | 58 ++++----- .../q76/simplified.txt | 30 ++--- .../q77/simplified.txt | 50 ++++---- .../q78/simplified.txt | 14 +-- .../q79/simplified.txt | 26 ++-- .../q8/simplified.txt | 26 ++-- .../q80/simplified.txt | 24 ++-- .../q81/simplified.txt | 28 ++--- .../q82/simplified.txt | 22 ++-- .../q83/simplified.txt | 44 +++---- .../q84/simplified.txt | 28 ++--- .../q85/simplified.txt | 42 +++---- .../q86/simplified.txt | 18 +-- .../q87/simplified.txt | 28 ++--- .../q88/simplified.txt | 104 +++++++-------- .../q89/simplified.txt | 24 ++-- .../q9/simplified.txt | 20 +-- .../q90/simplified.txt | 32 ++--- .../q91/simplified.txt | 42 +++---- .../q92/simplified.txt | 20 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 8 +- .../q95/simplified.txt | 12 +- .../q96/simplified.txt | 20 +-- .../q97/simplified.txt | 10 +- .../q98/simplified.txt | 18 +-- .../q99/simplified.txt | 28 ++--- .../q1/simplified.txt | 18 +-- .../q10/simplified.txt | 22 ++-- .../q11/simplified.txt | 52 ++++---- .../q12/simplified.txt | 16 +-- .../q13/simplified.txt | 28 ++--- .../q14a/simplified.txt | 54 ++++---- .../q14b/simplified.txt | 56 ++++----- .../q15/simplified.txt | 20 +-- .../q16/simplified.txt | 8 +- .../q17/simplified.txt | 44 +++---- .../q18/simplified.txt | 38 +++--- .../q19/simplified.txt | 28 ++--- .../q2/simplified.txt | 12 +- .../q20/simplified.txt | 16 +-- .../q21/simplified.txt | 22 ++-- .../q22/simplified.txt | 20 +-- .../q23a/simplified.txt | 38 +++--- .../q23b/simplified.txt | 44 +++---- .../q24a/simplified.txt | 14 +-- .../q24b/simplified.txt | 14 +-- .../q25/simplified.txt | 44 +++---- .../q26/simplified.txt | 28 ++--- .../q27/simplified.txt | 28 ++--- .../q28/simplified.txt | 12 +- .../q29/simplified.txt | 50 ++++---- .../q3/simplified.txt | 14 +-- .../q30/simplified.txt | 24 ++-- .../q31/simplified.txt | 56 ++++----- .../q32/simplified.txt | 18 +-- .../q33/simplified.txt | 44 +++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q36/simplified.txt | 22 ++-- .../q37/simplified.txt | 22 ++-- .../q38/simplified.txt | 28 ++--- .../q39a/simplified.txt | 34 ++--- .../q39b/simplified.txt | 34 ++--- .../q4/simplified.txt | 72 +++++------ .../q40/simplified.txt | 10 +- .../q41/simplified.txt | 6 +- .../q42/simplified.txt | 14 +-- .../q43/simplified.txt | 14 +-- .../q44/simplified.txt | 6 +- .../q45/simplified.txt | 28 ++--- .../q46/simplified.txt | 28 ++--- .../q47/simplified.txt | 20 +-- .../q48/simplified.txt | 24 ++-- .../q49/simplified.txt | 36 +++--- .../q5/simplified.txt | 52 ++++---- .../q50/simplified.txt | 26 ++-- .../q51/simplified.txt | 14 +-- .../q52/simplified.txt | 14 +-- .../q53/simplified.txt | 20 +-- .../q54/simplified.txt | 32 ++--- .../q55/simplified.txt | 14 +-- .../q56/simplified.txt | 44 +++---- .../q57/simplified.txt | 20 +-- .../q58/simplified.txt | 36 +++--- .../q59/simplified.txt | 10 +- .../q6/simplified.txt | 28 ++--- .../q60/simplified.txt | 44 +++---- .../q61/simplified.txt | 50 ++++---- .../q62/simplified.txt | 26 ++-- .../q63/simplified.txt | 20 +-- .../q64/simplified.txt | 32 ++--- .../q65/simplified.txt | 18 +-- .../q66/simplified.txt | 38 +++--- .../q67/simplified.txt | 22 ++-- .../q68/simplified.txt | 28 ++--- .../q69/simplified.txt | 22 ++-- .../q7/simplified.txt | 28 ++--- .../q70/simplified.txt | 22 ++-- .../q71/simplified.txt | 32 ++--- .../q72/simplified.txt | 50 ++++---- .../q73/simplified.txt | 26 ++-- .../q74/simplified.txt | 52 ++++---- .../q75/simplified.txt | 58 ++++----- .../q76/simplified.txt | 28 ++--- .../q77/simplified.txt | 38 +++--- .../q78/simplified.txt | 14 +-- .../q79/simplified.txt | 24 ++-- .../q8/simplified.txt | 26 ++-- .../q80/simplified.txt | 24 ++-- .../q81/simplified.txt | 24 ++-- .../q82/simplified.txt | 22 ++-- .../q83.ansi/simplified.txt | 38 +++--- .../q84/simplified.txt | 28 ++--- .../q85/simplified.txt | 40 +++--- .../q86/simplified.txt | 16 +-- .../q87/simplified.txt | 28 ++--- .../q88/simplified.txt | 104 +++++++-------- .../q89/simplified.txt | 22 ++-- .../q9/simplified.txt | 10 +- .../q90/simplified.txt | 32 ++--- .../q91/simplified.txt | 40 +++--- .../q92/simplified.txt | 18 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 8 +- .../q95/simplified.txt | 12 +- .../q96/simplified.txt | 20 +-- .../q97/simplified.txt | 10 +- .../q98/simplified.txt | 16 +-- .../q99/simplified.txt | 26 ++-- .../q10a/simplified.txt | 38 +++--- .../q11/simplified.txt | 60 ++++----- .../q12/simplified.txt | 18 +-- .../q14/simplified.txt | 58 ++++----- .../q14a/simplified.txt | 62 ++++----- .../q18a/simplified.txt | 118 +++++++++--------- .../q20/simplified.txt | 18 +-- .../q22/simplified.txt | 16 +-- .../q22a/simplified.txt | 22 ++-- .../q24/simplified.txt | 14 +-- .../q27a/simplified.txt | 60 ++++----- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q35a/simplified.txt | 38 +++--- .../q36a/simplified.txt | 24 ++-- .../q47/simplified.txt | 22 ++-- .../q49/simplified.txt | 42 +++---- .../q51a/simplified.txt | 18 +-- .../q57/simplified.txt | 22 ++-- .../q5a/simplified.txt | 58 ++++----- .../q6/simplified.txt | 30 ++--- .../q64/simplified.txt | 32 ++--- .../q67a/simplified.txt | 24 ++-- .../q70a/simplified.txt | 24 ++-- .../q72/simplified.txt | 50 ++++---- .../q74/simplified.txt | 60 ++++----- .../q75/simplified.txt | 58 ++++----- .../q77a/simplified.txt | 50 ++++---- .../q78/simplified.txt | 14 +-- .../q80a/simplified.txt | 24 ++-- .../q86a/simplified.txt | 18 +-- .../q98/simplified.txt | 18 +-- .../q10a/simplified.txt | 38 +++--- .../q11/simplified.txt | 52 ++++---- .../q12/simplified.txt | 16 +-- .../q14/simplified.txt | 56 ++++----- .../q14a/simplified.txt | 60 ++++----- .../q18a/simplified.txt | 108 ++++++++-------- .../q20/simplified.txt | 16 +-- .../q22/simplified.txt | 16 +-- .../q22a/simplified.txt | 20 +-- .../q24/simplified.txt | 14 +-- .../q27a/simplified.txt | 54 ++++---- .../q34/simplified.txt | 26 ++-- .../q35/simplified.txt | 22 ++-- .../q35a/simplified.txt | 36 +++--- .../q36a/simplified.txt | 22 ++-- .../q47/simplified.txt | 20 +-- .../q49/simplified.txt | 36 +++--- .../q51a/simplified.txt | 14 +-- .../q57/simplified.txt | 20 +-- .../q5a/simplified.txt | 52 ++++---- .../q6/simplified.txt | 28 ++--- .../q64/simplified.txt | 32 ++--- .../q67a/simplified.txt | 22 ++-- .../q70a/simplified.txt | 22 ++-- .../q72/simplified.txt | 50 ++++---- .../q74/simplified.txt | 52 ++++---- .../q75/simplified.txt | 58 ++++----- .../q77a/simplified.txt | 38 +++--- .../q78/simplified.txt | 14 +-- .../q80a/simplified.txt | 24 ++-- .../q86a/simplified.txt | 16 +-- .../q98/simplified.txt | 16 +-- 270 files changed, 4121 insertions(+), 4121 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt index dc0fb18f3..d6f75837b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt index dd1a52206..127a5a291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt @@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt index 905a35c8b..4c2d3f3d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt index 4de403664..98d995db0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt @@ -5,18 +5,18 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt index e3dfa631b..2257d398f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt @@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt index a03346372..b876fe4de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip,cs_sales_price] + CometHashAggregate [ca_zip,sum,cs_sales_price] CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt index 77aba376e..ae6bab279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [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,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt index 3d101857b..2519d58a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt @@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt index 18a69bcb4..163a31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index c7999d981..5d0658192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -13,16 +13,16 @@ WholeStageCodegen (7) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt index d805e3868..08088a386 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt index 52bd7a85e..bd81a3a7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt @@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt index bda583c17..7d36dc400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt @@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt index f818fd25f..e065a0a87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -81,12 +81,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -107,13 +107,13 @@ WholeStageCodegen (18) WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt index 54ee3dbde..2862f64cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt index 74a5e8777..1f4faaac5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt index cba306f68..a39fbbb79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt index 7fcbe967a..9c4dd659a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt @@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt index 47a4fe2c7..80c886708 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt @@ -15,9 +15,9 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -33,9 +33,9 @@ WholeStageCodegen (18) WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -51,9 +51,9 @@ WholeStageCodegen (18) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -69,9 +69,9 @@ WholeStageCodegen (18) WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -87,9 +87,9 @@ WholeStageCodegen (18) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -105,7 +105,7 @@ WholeStageCodegen (18) WholeStageCodegen (15) ColumnarToRow InputAdapter - CometHashAggregate [ss_list_price] + CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt index 2bcef6168..88b4ecd1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt @@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt index cc9c4edf3..bccda58a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt index f24e34c36..a80e0ca71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index a94a8a94d..a48c7ad26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -17,24 +17,24 @@ WholeStageCodegen (13) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -46,22 +46,22 @@ WholeStageCodegen (13) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -73,22 +73,22 @@ WholeStageCodegen (13) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -100,12 +100,12 @@ WholeStageCodegen (13) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -119,12 +119,12 @@ WholeStageCodegen (13) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -138,12 +138,12 @@ WholeStageCodegen (13) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt index b8df1e929..c3bfdfd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt index af2b7cb5d..7a3c2c153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt index 574a20fc0..4ad9b501b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt @@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index 7d28d5c10..0c709e4f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -50,14 +50,14 @@ WholeStageCodegen (5) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt index 048da153b..1a19c58ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt @@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt index 3e69a3341..e3b850ef5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt index d29a65bd4..5a060b85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt index 0a66507e1..232d65d72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt @@ -30,9 +30,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_net_profit] + CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -40,9 +40,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,ss_net_profit] + CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) @@ -69,7 +69,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt index 93e3eb05b..57defd961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt index d2e615f39..b6e5c469b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt index 6ebf6af07..5f628dbae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt @@ -5,16 +5,16 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_quantity] + CometHashAggregate [sum,ss_quantity] CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt index c4ea8fe24..537cba446 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt @@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt index 473b9cdd4..58adabcec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt index ce6005da6..988297f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt @@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt index 557dd3b4a..a0932c7cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt index 83e53bb66..b2199fb34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt index c6886735f..2193d875c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt index 2750a6ba2..fc38884c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt @@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt index 7fdead831..0b48046fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt index c630cad48..fb2b09b9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt index 2ed2bde44..fc7c21e94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt index e00d52dbb..6c1e6cd92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt @@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt index 89a080d85..824670101 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt index b76e7c9b3..7cfcb75da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt @@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_ext_sales_price] + CometHashAggregate [i_item_id,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cs_ext_sales_price] + CometHashAggregate [i_item_id,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ws_ext_sales_price] + CometHashAggregate [i_item_id,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt index 4ca1dd667..1b2af33b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt @@ -7,20 +7,20 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -62,18 +62,18 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_sales_price] + CometHashAggregate [sum,ss_ext_sales_price] CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt index c6b7e1834..0b4fc61a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt index 35e09ec7d..b2033c7a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt @@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt index 8de564ed1..6cb247973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt index d746739b3..72133811b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt @@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt index ef965d7d0..4dc4bcaf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt @@ -17,15 +17,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity] CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt index 43f44c9f7..c5f56a69c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt @@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt index c583ba8e8..f327d5b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt @@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt index 7cddbc640..030dfc4d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -58,15 +58,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index fd6777886..c604a8fa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -9,19 +9,19 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt index 2e6286d93..7c2a42ca3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt index 473eef9e4..e463296d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt @@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometUnion + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt index 590c59fdc..f9088f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt index b68a9474b..57598ec35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt index ac7b2c7e0..11f3a0ff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt @@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt index ef7d35e21..390c20b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,sr_return_quantity] + CometHashAggregate [i_item_id,sum,sr_return_quantity] CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,cr_return_quantity] + CometHashAggregate [i_item_id,sum,cr_return_quantity] CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,wr_return_quantity] + CometHashAggregate [i_item_id,sum,wr_return_quantity] CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt index ecae29c78..2313467ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt @@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt index 4218938c1..a6d1c25fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt @@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt index 507ac8a91..aa858c158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt @@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt index c54606f6e..5cb600551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt @@ -9,9 +9,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -24,9 +24,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -39,9 +39,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -54,9 +54,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -69,9 +69,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index b415eb5c4..232c174cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -9,23 +9,23 @@ WholeStageCodegen (3) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt index e2f498028..0681a64bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -36,15 +36,15 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt index c03b8be9a..3895cdc78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt @@ -18,12 +18,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt index adfe90bab..fc63929bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt @@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt index 26b5570b7..43bad5d70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_customer_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 @@ -47,8 +47,8 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id] ColumnarToRow InputAdapter CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index 19243e359..efd4b187d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt index fe6b3a625..240d88bb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -15,23 +15,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -44,23 +44,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -74,13 +74,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -95,13 +95,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 4bdb7ae0f..1640fa3b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt index ef54c7add..cc52bb323 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -7,16 +7,16 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,hd_dep_count] CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk,d_date_sk] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,22 +24,22 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt index 72aa4a054..d2ea2d830 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt @@ -20,19 +20,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 @@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 @@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #9 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #10 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #11 + CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #11 InputAdapter @@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 @@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 @@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt index 73badab8d..cd4817202 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt @@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 @@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 @@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #6 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 @@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt index b0fd9ebf1..a34cd9c1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -8,12 +8,12 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip] CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 + CometFilter [ca_address_sk,ca_state,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt index a55c182be..8935abb54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [cc_call_center_sk] - CometFilter [cc_county,cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt index 5c9726a22..4a634b069 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] + CometFilter [d_date_sk,d_quarter_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt index 561ad3d4f..c49bfc6c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt @@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag InputAdapter CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt index 852b61d1c..60caecf91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -8,31 +8,31 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,s_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_zip] #5 CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #6 - CometFilter [s_zip,s_store_sk] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometFilter [s_store_sk,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt index 334c6f0ea..535c61622 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -15,14 +15,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange #3 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 @@ -30,7 +30,7 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 @@ -46,5 +46,5 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_week_seq] + CometFilter [d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index dd1edb95f..5bfd9a907 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt index c6058d18f..2c8892b68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt @@ -9,27 +9,27 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date] CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id] CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - CometFilter [inv_warehouse_sk,inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange #5 - CometFilter [d_date,d_date_sk] + CometBroadcastExchange [d_date_sk,d_date] #5 + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt index d89167ec8..eb25d6d3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt @@ -9,12 +9,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] InputAdapter CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt index 1ed007bc7..412f27ffb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt @@ -28,7 +28,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -41,12 +41,12 @@ WholeStageCodegen (18) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -54,14 +54,14 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -83,10 +83,10 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -94,12 +94,12 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -109,11 +109,11 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt index 09a2bf7ce..d784ed47e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 @@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 @@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_date,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #8 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (6) @@ -84,10 +84,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk,d_date_sk] CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 @@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange #14 + CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter @@ -110,11 +110,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,c_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #10 + CometBroadcastExchange [c_customer_sk] #10 CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter @@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (10) @@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow InputAdapter - CometFilter [ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt index 8ebd45fd1..30091beb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -39,7 +39,7 @@ WholeStageCodegen (11) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 @@ -74,7 +74,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -85,7 +85,7 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #5 @@ -93,26 +93,26 @@ WholeStageCodegen (11) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_birth_country] + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_country,ca_zip] + CometFilter [ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index 41efffcb4..6335e50ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt index 67f335c66..b98e61704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,p_promo_sk] CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk,cd_demo_sk] + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt index 74c8a1ca2..4d4c824bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] InputAdapter CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt index a7a3f9537..2136886fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 @@ -35,7 +35,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #6 @@ -53,7 +53,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #9 @@ -71,7 +71,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #12 @@ -89,7 +89,7 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #15 @@ -107,5 +107,5 @@ WholeStageCodegen (18) ColumnarToRow InputAdapter CometProject [ss_list_price] - CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt index dbdfc8572..ee1f435d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt @@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc] CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] CometScan parquet spark_catalog.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 #2 @@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.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 #4 @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [cs_bill_customer_sk,cs_item_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk] #9 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #10 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 + CometFilter [i_item_sk,i_item_id,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt index 3dc9ede25..36bb17a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt index d5b8077ba..c4e59b49a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt @@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_addr_sk] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #9 @@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt index c004db9b7..9040179da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -19,22 +19,22 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_county] #5 CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter @@ -48,20 +48,20 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #9 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -75,20 +75,20 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #13 - CometFilter [d_qoy,d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter @@ -102,10 +102,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year,d_qoy] #4 @@ -121,10 +121,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #9 @@ -140,10 +140,10 @@ WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk,d_year,d_qoy] #13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index 8cc26d8f4..b6ba1fbab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -38,13 +38,13 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_ext_discount_amt] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt index 84143e015..bf0ef7ae4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_manufact_id,i_manufact_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_manufact_id] #7 CometProject [i_manufact_id] - CometFilter [i_category] + CometFilter [i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index eefd38343..e1f7f9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt index efe0b0b4e..9deb51342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 5bdf60081..58d617430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt index c3fcd79f0..efd86d5ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt index ff321c523..d16db5c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -14,12 +14,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -52,12 +52,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt index ff321c523..d16db5c8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -14,12 +14,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,17 +27,17 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_moy] #6 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 @@ -52,12 +52,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy] CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -65,11 +65,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk,d_moy] #10 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt index 0918da575..a412a54d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt @@ -18,23 +18,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -47,23 +47,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -77,13 +77,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -98,13 +98,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [cs_bill_customer_sk] + CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15 + CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 @@ -120,13 +120,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #18 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -141,13 +141,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #21 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt index 10e0735b4..11c0201a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt @@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [cs_warehouse_sk,cs_item_sk] + CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [w_warehouse_sk] + CometFilter [w_warehouse_sk,w_state] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #6 @@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] ColumnarToRow InputAdapter CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt index e31217066..457d441ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name] ColumnarToRow InputAdapter CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact] + CometFilter [i_manufact_id,i_manufact,i_product_name] CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_manufact] + CometHashAggregate [i_manufact,count] CometProject [i_manufact] - CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt index e3967a972..d6e3733ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt index 63945612a..f21d846c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed ColumnarToRow InputAdapter CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name] CometProject [d_day_name,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange #2 - CometFilter [ss_store_sk] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt index 35a3e9efa..12c8e0af9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt @@ -35,7 +35,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_store_sk,ss_addr_sk] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] InputAdapter @@ -45,7 +45,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_store_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (10) @@ -75,7 +75,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index c376c4fcd..f60fdb18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip] CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - CometFilter [ws_bill_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #4 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 + CometFilter [ca_address_sk,ca_city,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 @@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] ColumnarToRow InputAdapter CometProject [i_item_id] - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt index 718557c10..9ce3a98eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 66eb5a4cb..1db8e7528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra ColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt index 30f9822ab..23e083535 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt @@ -7,14 +7,14 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [ss_quantity] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state] CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk] + CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,19 +22,19 @@ WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk,ca_state] #5 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index d1480ef10..bd3b21cdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt index 3dc3cca6c..ceb3e9213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -52,21 +52,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #7 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] @@ -77,23 +77,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #10 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt index cd8a8ff7f..aa3f7f2cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt @@ -8,17 +8,17 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk] CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2 + CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index 9ab7f18e7..f19312ca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -32,8 +32,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (10) @@ -69,8 +69,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt index d0b9da882..d7661b7be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] ColumnarToRow InputAdapter CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [d_year,ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk,d_year] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index a04777412..b90cb42d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -17,16 +17,16 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manufact_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] ColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt index 32e9a92a7..a8df48770 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [c_customer_sk,c_current_addr_sk] CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [item_sk,i_item_sk] - CometUnion + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk] #5 CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #7 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [ss_customer_sk] + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 @@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [(d_month_seq + 1)] #3 ReusedSubquery [(d_month_seq + 3)] #4 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] @@ -85,7 +85,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 1)] CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (2) @@ -97,7 +97,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] InputAdapter CometHashAggregate [(d_month_seq + 3)] CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter BroadcastExchange #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt index a404b9b37..2461ab830 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -8,16 +8,16 @@ TakeOrderedAndProject [ext_price,brand_id,brand] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_brand_id,i_brand] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #2 - CometFilter [ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt index a5cb59e57..c4ce35e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_color] + CometFilter [i_item_id,i_color] CometScan parquet spark_catalog.default.item [i_item_id,i_color] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index ee91816c9..b64c2efdc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ ColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt index 959d98ba9..0f5a783b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -13,10 +13,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,12 +24,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometFilter [d_week_seq] + CometFilter [d_date,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] Subquery #2 @@ -37,15 +37,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [i_item_sk,i_item_id] #4 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -60,10 +60,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 @@ -80,10 +80,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt index 55cb85ddb..58bae46d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -14,18 +14,18 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name] + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt index a7c65f50a..febbfd32f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 @@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 @@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state] HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometFilter [i_category] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index cf226bb14..51025cf34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -16,12 +16,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_addr_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [ca_address_sk] #5 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #6 - CometBroadcastHashJoin [i_item_id,i_item_id] - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #7 + CometBroadcastExchange [i_item_id] #7 CometProject [i_item_id] - CometFilter [i_category] + CometFilter [i_item_id,i_category] CometScan parquet spark_catalog.default.item [i_item_id,i_category] WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] @@ -56,12 +56,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -76,12 +76,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_addr_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt index db13f94bd..8d39d9e4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -9,18 +9,18 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk] CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,30 +28,30 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #3 + CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange #4 + CometBroadcastExchange [p_promo_sk] #4 CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 + CometBroadcastExchange [i_item_sk] #8 CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #9 @@ -64,16 +64,16 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk] CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [s_store_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt index b25008e64..b9e35e282 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt @@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ColumnarToRow InputAdapter CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk] CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index a8eb9d7c6..e84d3c2a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -17,16 +17,16 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometProject [i_item_sk,i_manager_id] - CometFilter [i_category,i_class,i_brand,i_item_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index d972e0082..3b5e4f19e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt index 1e34ce22f..84d23af42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes BroadcastHashJoin [s_store_sk,ss_store_sk] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 @@ -22,8 +22,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #6 @@ -61,8 +61,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt index 7d6c85af3..880c38bbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt @@ -16,35 +16,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange #5 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk] #6 CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] + CometFilter [t_time_sk,t_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange #7 + CometBroadcastExchange [sm_ship_mode_sk] #7 CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] @@ -55,14 +55,14 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat ColumnarToRow InputAdapter CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk] CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy] CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt index 0386feff9..46d9cf4ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt @@ -20,12 +20,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt index 1ecde47e8..78e1b1cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] + CometFilter [s_store_sk,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt index de2d5eeda..c9cc4959b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt index 0429448f5..a27daca36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk] CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #5 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [p_promo_sk] #6 CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index c962534aa..4a6dffc08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #6 @@ -60,13 +60,13 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #8 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #8 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index c732d903f..a4be53684 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -11,17 +11,17 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [time_sk,t_time_sk] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,sold_item_sk] - CometBroadcastExchange #3 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion + CometUnion [ext_price,sold_item_sk,time_sk] CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_sold_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -29,25 +29,25 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_item_sk,cs_sold_time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_sold_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #6 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index 060c3e153..c91c4cf47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt index ea253444c..a7aa864ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt index 34866bc37..27a4dcb67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt index 6f8c28004..cfcf6a99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -7,32 +7,32 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] ColumnarToRow InputAdapter - CometUnion + CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometFilter [i_item_sk,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange #3 - CometFilter [d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_ship_customer_sk,ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_ship_addr_sk,cs_item_sk] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedExchange [i_item_sk,i_category] #2 ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt index 2989455d9..91786374e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -19,10 +19,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -84,7 +84,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -99,14 +99,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #12 + CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -120,10 +120,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt index 280687e30..13a8aecff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt index 5fcbcfed3..f2883e450 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -10,12 +10,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city] CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_dow,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk,s_city] #4 CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] + CometFilter [s_store_sk,s_number_employees,s_city] CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange #5 + CometBroadcastExchange [hd_demo_sk] #5 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index adf555417..637f3b7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_qoy,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 - CometFilter [s_store_sk,s_zip] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 + CometFilter [s_store_sk,s_store_name,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #5 @@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_zip] + CometHashAggregate [ca_zip,count] CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometFilter [ca_address_sk,ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange #9 + CometBroadcastExchange [c_current_addr_sk] #9 CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt index 7e257bdc6..cdc8dc486 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 @@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 @@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] @@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #7 @@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #4 @@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt index 3daee59f8..991c00de9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt @@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 + CometBroadcastExchange [ca_address_sk,ca_state] #4 CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter @@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ColumnarToRow InputAdapter CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #3 @@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st WholeStageCodegen (5) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt index a4d96e6f1..71a1c8e17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] InputAdapter CometHashAggregate [i_item_id,i_item_desc,i_current_price] CometProject [i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometBroadcastExchange #2 + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2 CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange #3 + CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt index 7b7e907ee..8cd3f3f45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt @@ -12,10 +12,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [sr_return_quantity,i_item_id] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk] CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [sr_item_sk,i_item_sk] - CometFilter [sr_item_sk] + CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date] #3 CometProject [d_date] - CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_week_seq] #4 CometProject [d_week_seq] - CometFilter [d_date] + CometFilter [d_date,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [i_item_sk,i_item_id] #5 CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometBroadcastHashJoin [d_date,d_date] - CometFilter [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 InputAdapter @@ -54,10 +54,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [cr_return_quantity,i_item_id] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk] CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [cr_item_sk,i_item_sk] - CometFilter [cr_item_sk] + CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 @@ -73,10 +73,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ColumnarToRow InputAdapter CometProject [wr_return_quantity,i_item_id] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk] CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] - CometBroadcastHashJoin [wr_item_sk,i_item_sk] - CometFilter [wr_item_sk] + CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id] + CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index fe7f7a207..646285a08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] Project [c_customer_id,c_last_name,c_first_name] ColumnarToRow InputAdapter - CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange #1 + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange #2 + CometBroadcastExchange [ca_address_sk] #2 CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] + CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 CometFilter [hd_demo_sk,hd_income_band_sk] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [ib_income_band_sk] #5 CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt index ad94ef82c..0d350bd59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt @@ -8,21 +8,21 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc] CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk] CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - CometBroadcastExchange #2 - CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2 + CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] - CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #6 + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk,ca_state] #7 CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange #8 + CometBroadcastExchange [d_date_sk] #8 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [r_reason_sk] + CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index ea5dce5b6..e4d028f2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -17,10 +17,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt index cfac83844..9d667265c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -15,10 +15,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,14 +26,14 @@ WholeStageCodegen (7) ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #5 - CometFilter [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 + CometFilter [c_customer_sk,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 @@ -46,10 +46,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [cs_bill_customer_sk,d_date] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] + CometFilter [cs_bill_customer_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 @@ -65,10 +65,10 @@ WholeStageCodegen (7) InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] CometProject [ws_bill_customer_sk,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_bill_customer_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt index b846d25d8..d3af10d19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt @@ -12,27 +12,27 @@ WholeStageCodegen (16) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 @@ -43,20 +43,20 @@ WholeStageCodegen (16) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -68,20 +68,20 @@ WholeStageCodegen (16) WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #10 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -93,20 +93,20 @@ WholeStageCodegen (16) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #13 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -118,20 +118,20 @@ WholeStageCodegen (16) WholeStageCodegen (8) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #16 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -143,20 +143,20 @@ WholeStageCodegen (16) WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #19 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -168,20 +168,20 @@ WholeStageCodegen (16) WholeStageCodegen (12) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #22 + CometBroadcastExchange [t_time_sk] #22 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 InputAdapter @@ -193,19 +193,19 @@ WholeStageCodegen (16) WholeStageCodegen (14) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #25 + CometBroadcastExchange [t_time_sk] #25 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 52e066fe7..2bfff90d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -17,15 +17,15 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_category,i_class,i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt index fdd3bd293..581739cc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt @@ -11,7 +11,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 @@ -41,7 +41,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 @@ -56,7 +56,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 @@ -71,7 +71,7 @@ WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity] + CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt index 50c8494fb..44159cc73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt @@ -7,27 +7,27 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange #4 + CometBroadcastExchange [wp_web_page_sk] #4 CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] + CometFilter [wp_web_page_sk,wp_char_count] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 @@ -38,19 +38,19 @@ WholeStageCodegen (4) WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange #7 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] + CometFilter [t_time_sk,t_hour] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt index 7e9b7b8ba..84f1190d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -11,21 +11,21 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometFilter [cc_call_center_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange #3 - CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -33,23 +33,23 @@ WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange #7 + CometBroadcastExchange [ca_address_sk] #7 CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange #8 - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange #9 + CometBroadcastExchange [hd_demo_sk] #9 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index bcc1ef1b5..aa24a049b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -11,8 +11,8 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -20,11 +20,11 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #3 + CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] + CometFilter [i_item_sk,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 @@ -38,13 +38,13 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_ext_discount_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt index 3ec7ac7b6..a8eb231b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] InputAdapter BroadcastExchange #4 @@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] ColumnarToRow InputAdapter CometProject [r_reason_sk] - CometFilter [r_reason_desc,r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt index 34ddde768..d54c9e0c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -26,7 +26,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -54,7 +54,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -62,7 +62,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -70,5 +70,5 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt index 5b699890c..178c25f66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -25,7 +25,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] InputAdapter WholeStageCodegen (7) @@ -40,7 +40,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_order_number,ws_warehouse_sk] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -60,7 +60,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [wr_order_number] - CometFilter [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] InputAdapter WholeStageCodegen (15) @@ -82,7 +82,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 @@ -90,7 +90,7 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 @@ -98,5 +98,5 @@ WholeStageCodegen (21) ColumnarToRow InputAdapter CometProject [web_site_sk] - CometFilter [web_company_name,web_site_sk] + CometFilter [web_site_sk,web_company_name] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt index 614915226..e9d33a7f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt @@ -5,25 +5,25 @@ WholeStageCodegen (2) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate + CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [hd_demo_sk] #2 CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange #3 + CometBroadcastExchange [t_time_sk] #3 CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange #4 + CometBroadcastExchange [s_store_sk] #4 CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] + CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt index 24e6dceef..41f3e579f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt @@ -17,7 +17,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [ss_customer_sk,ss_item_sk] CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -25,11 +25,11 @@ WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter WholeStageCodegen (4) @@ -42,7 +42,7 @@ WholeStageCodegen (6) InputAdapter CometHashAggregate [cs_bill_customer_sk,cs_item_sk] CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 84d277886..a92d99be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -20,10 +20,10 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -31,12 +31,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt index da4ef22e0..25ff3a2fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt @@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days , ColumnarToRow InputAdapter CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk] CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange #2 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #3 - CometFilter [sm_ship_mode_sk] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometFilter [sm_ship_mode_sk,sm_type] CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange #4 - CometFilter [cc_call_center_sk] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index ecc421bd5..25010c5eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index 545f0ecec..a8b8a9cce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index 799f74a36..aa49638c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter @@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index 45061c290..7ef901a39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [quantity,list_price] - CometUnion + CometHashAggregate [sum,count,quantity,list_price] + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index d4deedf0a..f854b2b31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index 513e6f979..e460ce5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index 6ab6c4fd7..30307e528 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index 371254b08..de75d46ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt @@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt index e3a91e471..8d73022ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt index 9c35ee397..256fddfc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index a85302cc5..eeeb5ba36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index 8d7b158d0..0e6b65b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index 78dd29c2a..e9c4d46ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store WholeStageCodegen (11) ColumnarToRow InputAdapter - CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index c2c8e089f..968f7e1af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index dd66c9582..1428c7ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (3) ColumnarToRow InputAdapter - CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index 9607ab887..cfb74b3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_current_price] - CometFilter [i_category] + CometHashAggregate [i_category,sum,count,i_current_price] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index aeb5b67da..5838786d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -20,14 +20,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity] CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index 74305dd6e..e76a524db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -65,16 +65,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_state,ss_net_profit] + CometHashAggregate [s_state,sum,ss_net_profit] CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index 283fe5bde..e5cf23f50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (4) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index 752b8c854..34972d290 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) ColumnarToRow InputAdapter - CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) ColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (7) ColumnarToRow InputAdapter - CometHashAggregate [cr_return_amount,cr_net_loss] + CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (9) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (10) ColumnarToRow InputAdapter - CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index 0b7ad4726..5adee59be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_category,i_class,ws_net_paid] + CometHashAggregate [i_category,i_class,sum,ws_net_paid] CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 6484c2dcb..64b162608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -17,12 +17,12 @@ WholeStageCodegen (5) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt index 520edc88d..22c208f2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt @@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 + CometBroadcastExchange [ca_address_sk] #6 CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] + CometFilter [ca_address_sk,ca_county] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt index 4b9e60e18..d7c6ef3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 3d0576653..07c91b94d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt index 73badab8d..cd4817202 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt @@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [d_date_sk] #10 @@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #2 @@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 @@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 @@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 @@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #7 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #8 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange #9 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #10 InputAdapter @@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 @@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #16 @@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_week_seq,d_date_sk] + CometFilter [d_date_sk,d_week_seq] ReusedSubquery [d_week_seq] #6 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] Subquery #6 @@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ColumnarToRow InputAdapter CometProject [d_week_seq] - CometFilter [d_year,d_moy,d_dom] + CometFilter [d_week_seq,d_year,d_moy,d_dom] CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt index 6e0ac9858..3dbd7bc33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt @@ -26,14 +26,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [quantity,list_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometUnion + CometUnion [quantity,list_price] CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #12 CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #16 @@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #17 + CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #17 @@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ss_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 @@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow InputAdapter - CometFilter [i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 @@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter CometHashAggregate [brand_id,class_id,category_id] CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 @@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange #11 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange #12 + CometBroadcastExchange [d_date_sk] #12 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [d_date_sk] #12 InputAdapter @@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ColumnarToRow InputAdapter CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 @@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #5 @@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [cs_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_item_sk,ss_item_sk] ColumnarToRow InputAdapter - CometFilter [ws_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt index 9f43cdc72..509d4be06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt @@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange #4 + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange #5 + CometBroadcastExchange [cd_demo_sk] #5 CometFilter [cd_demo_sk] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange #6 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #8 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] @@ -60,25 +60,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #10 - CometFilter [ca_state,ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -91,26 +91,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #12 + CometBroadcastExchange [ca_address_sk,ca_country] #12 CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -123,26 +123,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange #14 + CometBroadcastExchange [ca_address_sk] #14 CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 @@ -155,18 +155,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ColumnarToRow InputAdapter CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk,cd_dep_count] #3 @@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 ReusedExchange [ca_address_sk] #14 ReusedExchange [d_date_sk] #7 - CometBroadcastExchange #16 + CometBroadcastExchange [i_item_sk] #16 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index da567f687..cd6bcfd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #4 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt index 415b430f8..0e864ab70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt index db81d1ba5..fc4778a22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt @@ -11,12 +11,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #3 + CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #4 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange #5 + CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] WholeStageCodegen (5) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt index 7024f439f..5b658f1d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt @@ -43,7 +43,7 @@ WholeStageCodegen (12) WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 @@ -78,7 +78,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] InputAdapter WholeStageCodegen (4) @@ -89,7 +89,7 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter BroadcastExchange #6 @@ -97,26 +97,26 @@ WholeStageCodegen (12) ColumnarToRow InputAdapter CometProject [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_market_id,s_store_sk,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) ColumnarToRow InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ca_address_sk,ca_country,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt index b09f5af82..50e246e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt @@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 @@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #3 + CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 - CometFilter [s_state,s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometFilter [i_item_sk,i_item_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] @@ -49,21 +49,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #8 + CometBroadcastExchange [s_store_sk] #8 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 WholeStageCodegen (6) @@ -75,19 +75,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ColumnarToRow InputAdapter CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [cd_demo_sk] #3 ReusedExchange [d_date_sk] #4 ReusedExchange [s_store_sk] #8 - CometBroadcastExchange #10 + CometBroadcastExchange [i_item_sk] #10 CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index 970562e5c..814e543ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -12,14 +12,14 @@ WholeStageCodegen (4) WholeStageCodegen (1) ColumnarToRow InputAdapter - CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,24 +27,24 @@ WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_dom,d_year,d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange #5 + CometBroadcastExchange [s_store_sk] #5 CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] + CometFilter [s_store_sk,s_county] CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange #6 + CometBroadcastExchange [hd_demo_sk] #6 CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) ColumnarToRow InputAdapter - CometFilter [c_customer_sk] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt index a6b4add1f..0b55e23ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt @@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 @@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 @@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ WholeStageCodegen (3) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) ColumnarToRow InputAdapter - CometFilter [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt index a95475f7b..79ead0464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt @@ -8,17 +8,17 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange #2 + CometBroadcastExchange [ss_customer_sk] #2 CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange #5 - CometUnion + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - CometBroadcastExchange #6 - CometFilter [ca_address_sk] + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #7 - CometFilter [cd_demo_sk] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index 485fdc3e2..59c22c335 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange #7 + CometBroadcastExchange [s_store_sk] #7 CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index a480db5ea..61f4976b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, ColumnarToRow InputAdapter CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,ss_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [ss_item_sk,ss_store_sk] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 CometFilter [s_store_sk,s_store_name,s_company_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index d1480ef10..bd3b21cdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - CometBroadcastExchange #4 + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] @@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #9 + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] ReusedExchange [d_date_sk] #6 WholeStageCodegen (15) @@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #12 + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index 892a3fc78..47ad5e34f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -44,8 +44,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 @@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange #7 + CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_date,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #8 @@ -98,8 +98,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_date] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index b038aedf0..daeafbf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m ColumnarToRow InputAdapter CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,cs_item_sk] - CometFilter [i_item_sk,i_category,i_brand] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometFilter [i_item_sk,i_brand,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange #3 - CometFilter [cs_item_sk,cs_call_center_sk] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_moy,d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #5 - CometFilter [d_year,d_moy,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 CometFilter [cc_call_center_sk,cc_name] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt index 34ad132c0..c3b00538b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,s_store_sk] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -59,21 +59,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #8 - CometFilter [cp_catalog_page_sk] + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] @@ -84,25 +84,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [date_sk,d_date_sk] - CometUnion + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #10 + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #11 - CometFilter [web_site_sk] + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt index a7c65f50a..febbfd32f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt @@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometFilter [ca_address_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometFilter [ca_address_sk,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange #2 - CometFilter [c_current_addr_sk,c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange #3 - CometFilter [ss_customer_sk,ss_item_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] Subquery #2 @@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state] InputAdapter CometHashAggregate [d_month_seq] CometProject [d_month_seq] - CometFilter [d_year,d_moy] + CometFilter [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] ReusedSubquery [d_month_seq] #2 @@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow InputAdapter - CometFilter [i_current_price,i_category,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #8 @@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state] HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter - CometFilter [i_category] + CometFilter [i_current_price,i_category] CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 2a0bc5bce..512a74f9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -54,19 +54,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #4 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #4 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (8) @@ -89,7 +89,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] InputAdapter WholeStageCodegen (6) @@ -100,7 +100,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] CometScan parquet spark_catalog.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] #5 @@ -116,14 +116,14 @@ WholeStageCodegen (52) WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [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] + CometFilter [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] CometScan parquet spark_catalog.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 (12) ColumnarToRow InputAdapter - CometFilter [d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #11 @@ -157,7 +157,7 @@ WholeStageCodegen (52) WholeStageCodegen (19) ColumnarToRow InputAdapter - CometFilter [ca_address_sk] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] CometScan parquet spark_catalog.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 @@ -176,7 +176,7 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [i_item_sk,i_product_name] - CometFilter [i_current_price,i_color,i_item_sk] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (50) @@ -227,19 +227,19 @@ WholeStageCodegen (52) ColumnarToRow InputAdapter CometProject [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] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - CometBroadcastExchange #20 - CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometBroadcastHashJoin [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,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [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] #20 + CometFilter [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] CometScan parquet spark_catalog.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) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter WholeStageCodegen (33) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt index f68ac8dfc..cc9627e89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt @@ -22,12 +22,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ ColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #4 + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange #5 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index a7c639a49..67f4841b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #6 @@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 @@ -67,14 +67,14 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count ColumnarToRow InputAdapter CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange #9 - CometFilter [s_store_sk] + CometBroadcastExchange [s_store_sk,s_state] #9 + CometFilter [s_store_sk,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [d_date_sk] #5 WholeStageCodegen (13) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt index bea1fd4a1..6cba2d0e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt @@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometBroadcastHashJoin [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,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [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] CometScan parquet spark_catalog.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 @@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #4 - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange #5 - CometFilter [w_warehouse_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange #7 + CometBroadcastExchange [cd_demo_sk] #7 CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange #8 + CometBroadcastExchange [hd_demo_sk] #8 CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange #9 + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange #10 - CometFilter [d_week_seq,d_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 @@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt index a2cb15b4e..4360a9555 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #2 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #4 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 @@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #7 - CometFilter [ss_customer_sk] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #9 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #10 @@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #12 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 @@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ColumnarToRow InputAdapter CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometFilter [c_customer_sk,c_customer_id] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange #15 - CometFilter [ws_bill_customer_sk] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt index fb78d64b1..2aecc9e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange #6 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange #7 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] @@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] @@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter WholeStageCodegen (38) @@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,i_item_sk] - CometFilter [cs_item_sk] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange #18 - CometFilter [d_year,d_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (23) @@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 @@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i ColumnarToRow InputAdapter CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt index 141e72805..db4e10247 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -26,10 +26,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #6 + CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter @@ -57,10 +57,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -79,7 +79,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -91,7 +91,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 @@ -106,14 +106,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - CometBroadcastExchange #13 + CometBroadcastExchange [wp_web_page_sk] #13 CometFilter [wp_web_page_sk] CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter @@ -127,10 +127,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt index 49bd173f6..b8e6e0605 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_customer_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [d_year,d_date_sk] + CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (4) @@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_ticket_number,sr_item_sk] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (8) ColumnarToRow InputAdapter - CometFilter [ws_item_sk,ws_bill_customer_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_order_number,wr_item_sk] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 @@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cs_item_sk,cs_bill_customer_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_order_number,cr_item_sk] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt index 34e47dcba..0793067df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (1) ColumnarToRow InputAdapter - CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 @@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (4) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (6) ColumnarToRow InputAdapter - CometFilter [s_store_sk] + CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [i_item_sk] - CometFilter [i_current_price,i_item_sk] + CometFilter [i_item_sk,i_current_price] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 @@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [p_promo_sk] - CometFilter [p_channel_tv,p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (20) HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] @@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) ColumnarToRow InputAdapter - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (16) ColumnarToRow InputAdapter - CometFilter [cp_catalog_page_sk] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 @@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) ColumnarToRow InputAdapter - CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 InputAdapter @@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] InputAdapter ReusedExchange [d_date_sk] #5 @@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (26) ColumnarToRow InputAdapter - CometFilter [web_site_sk] + CometFilter [web_site_sk,web_site_id] CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index 7edc6b80b..8996119f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -24,10 +24,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #5 + CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] + CometFilter [d_date_sk,d_month_seq] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange #6 - CometFilter [i_item_sk] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometFilter [i_item_sk,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index 2dd86f85b..3fc9eb261 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -19,10 +19,10 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,i_item_sk] - CometFilter [ss_item_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 @@ -30,12 +30,12 @@ WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange #5 - CometFilter [i_category,i_item_sk] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange #6 + CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] + CometFilter [d_date_sk,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]