diff --git a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index 5198c6dd2fdf0..5d9b5be311952 100644 --- a/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -27,6 +27,10 @@ object ExpressionSet { expressions.foreach(set.add) set } + + def apply(): ExpressionSet = { + new ExpressionSet() + } } /** @@ -53,46 +57,102 @@ object ExpressionSet { * This is consistent with how we define `semanticEquals` between two expressions. */ class ExpressionSet protected( - protected val baseSet: mutable.Set[Expression] = new mutable.HashSet, - protected val originals: mutable.Buffer[Expression] = new ArrayBuffer) - extends Set[Expression] { + private val baseSet: mutable.Set[Expression] = new mutable.HashSet, + private val originals: mutable.Buffer[Expression] = new ArrayBuffer) + extends Iterable[Expression] { // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. protected def add(e: Expression): Unit = { if (!e.deterministic) { originals += e - } else if (!baseSet.contains(e.canonicalized) ) { + } else if (!baseSet.contains(e.canonicalized)) { baseSet.add(e.canonicalized) originals += e } } - override def contains(elem: Expression): Boolean = baseSet.contains(elem.canonicalized) + protected def remove(e: Expression): Unit = { + if (e.deterministic) { + baseSet --= baseSet.filter(_ == e.canonicalized) + originals --= originals.filter(_.canonicalized == e.canonicalized) + } + } + + def contains(elem: Expression): Boolean = baseSet.contains(elem.canonicalized) + + override def filter(p: Expression => Boolean): ExpressionSet = { + val newBaseSet = baseSet.filter(e => p(e.canonicalized)) + val newOriginals = originals.filter(e => p(e.canonicalized)) + new ExpressionSet(newBaseSet, newOriginals) + } + + override def filterNot(p: Expression => Boolean): ExpressionSet = { + val newBaseSet = baseSet.filterNot(e => p(e.canonicalized)) + val newOriginals = originals.filterNot(e => p(e.canonicalized)) + new ExpressionSet(newBaseSet, newOriginals) + } - override def +(elem: Expression): ExpressionSet = { - val newSet = new ExpressionSet(baseSet.clone(), originals.clone()) + def +(elem: Expression): ExpressionSet = { + val newSet = clone() newSet.add(elem) newSet } - override def ++(elems: GenTraversableOnce[Expression]): ExpressionSet = { - val newSet = new ExpressionSet(baseSet.clone(), originals.clone()) + def ++(elems: GenTraversableOnce[Expression]): ExpressionSet = { + val newSet = clone() elems.foreach(newSet.add) newSet } - override def -(elem: Expression): ExpressionSet = { - if (elem.deterministic) { - val newBaseSet = baseSet.clone().filterNot(_ == elem.canonicalized) - val newOriginals = originals.clone().filterNot(_.canonicalized == elem.canonicalized) - new ExpressionSet(newBaseSet, newOriginals) - } else { - new ExpressionSet(baseSet.clone(), originals.clone()) - } + def -(elem: Expression): ExpressionSet = { + val newSet = clone() + newSet.remove(elem) + newSet + } + + def --(elems: GenTraversableOnce[Expression]): ExpressionSet = { + val newSet = clone() + elems.foreach(newSet.remove) + newSet } - override def iterator: Iterator[Expression] = originals.iterator + def map(f: Expression => Expression): ExpressionSet = { + val newSet = new ExpressionSet() + this.iterator.foreach(elem => newSet.add(f(elem))) + newSet + } + + def flatMap(f: Expression => Iterable[Expression]): ExpressionSet = { + val newSet = new ExpressionSet() + this.iterator.foreach(f(_).foreach(newSet.add)) + newSet + } + + def iterator: Iterator[Expression] = originals.iterator + + def union(that: ExpressionSet): ExpressionSet = { + val newSet = clone() + that.iterator.foreach(newSet.add) + newSet + } + + def subsetOf(that: ExpressionSet): Boolean = this.iterator.forall(that.contains) + + def intersect(that: ExpressionSet): ExpressionSet = this.filter(that.contains) + + def diff(that: ExpressionSet): ExpressionSet = this -- that + + def apply(elem: Expression): Boolean = this.contains(elem) + + override def equals(obj: Any): Boolean = obj match { + case other: ExpressionSet => this.baseSet == other.baseSet + case _ => false + } + + override def hashCode(): Int = baseSet.hashCode() + + override def clone(): ExpressionSet = new ExpressionSet(baseSet.clone(), originals.clone()) /** * Returns a string containing both the post [[Canonicalize]] expressions and the original diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 038ebb2037a27..e43a9a8c46143 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -37,7 +37,11 @@ object AttributeSet { val empty = apply(Iterable.empty) /** Constructs a new [[AttributeSet]] that contains a single [[Attribute]]. */ - def apply(a: Attribute): AttributeSet = new AttributeSet(Set(new AttributeEquals(a))) + def apply(a: Attribute): AttributeSet = { + val baseSet = new mutable.LinkedHashSet[AttributeEquals] + baseSet += new AttributeEquals(a) + new AttributeSet(baseSet) + } /** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */ def apply(baseSet: Iterable[Expression]): AttributeSet = { @@ -47,7 +51,7 @@ object AttributeSet { /** Constructs a new [[AttributeSet]] given a sequence of [[AttributeSet]]s. */ def fromAttributeSets(sets: Iterable[AttributeSet]): AttributeSet = { val baseSet = sets.foldLeft(new mutable.LinkedHashSet[AttributeEquals]())( _ ++= _.baseSet) - new AttributeSet(baseSet.toSet) + new AttributeSet(baseSet) } } @@ -62,7 +66,7 @@ object AttributeSet { * and also makes doing transformations hard (we always try keep older trees instead of new ones * when the transformation was a no-op). */ -class AttributeSet private (val baseSet: Set[AttributeEquals]) +class AttributeSet private (private val baseSet: mutable.LinkedHashSet[AttributeEquals]) extends Iterable[Attribute] with Serializable { override def hashCode: Int = baseSet.hashCode() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index c450ea891a612..a64e8bcd68175 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, ExpressionSet, PredicateHelper} import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -75,18 +75,18 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { * Extracts items of consecutive inner joins and join conditions. * This method works for bushy trees and left/right deep trees. */ - private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = { + private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], ExpressionSet) = { plan match { case Join(left, right, _: InnerLike, Some(cond), JoinHint.NONE) => val (leftPlans, leftConditions) = extractInnerJoins(left) val (rightPlans, rightConditions) = extractInnerJoins(right) - (leftPlans ++ rightPlans, splitConjunctivePredicates(cond).toSet ++ - leftConditions ++ rightConditions) + (leftPlans ++ rightPlans, leftConditions ++ rightConditions ++ + splitConjunctivePredicates(cond)) case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), JoinHint.NONE)) if projectList.forall(_.isInstanceOf[Attribute]) => extractInnerJoins(j) case _ => - (Seq(plan), Set()) + (Seq(plan), ExpressionSet()) } } @@ -143,7 +143,7 @@ object JoinReorderDP extends PredicateHelper with Logging { def search( conf: SQLConf, items: Seq[LogicalPlan], - conditions: Set[Expression], + conditions: ExpressionSet, output: Seq[Attribute]): LogicalPlan = { val startTime = System.nanoTime() @@ -151,7 +151,7 @@ object JoinReorderDP extends PredicateHelper with Logging { // Create the initial plans: each plan is a single item with zero cost. val itemIndex = items.zipWithIndex val foundPlans = mutable.Buffer[JoinPlanMap](itemIndex.map { - case (item, id) => Set(id) -> JoinPlan(Set(id), item, Set.empty, Cost(0, 0)) + case (item, id) => Set(id) -> JoinPlan(Set(id), item, ExpressionSet(), Cost(0, 0)) }.toMap) // Build filters from the join graph to be used by the search algorithm. @@ -194,7 +194,7 @@ object JoinReorderDP extends PredicateHelper with Logging { private def searchLevel( existingLevels: Seq[JoinPlanMap], conf: SQLConf, - conditions: Set[Expression], + conditions: ExpressionSet, topOutput: AttributeSet, filters: Option[JoinGraphInfo]): JoinPlanMap = { @@ -255,7 +255,7 @@ object JoinReorderDP extends PredicateHelper with Logging { oneJoinPlan: JoinPlan, otherJoinPlan: JoinPlan, conf: SQLConf, - conditions: Set[Expression], + conditions: ExpressionSet, topOutput: AttributeSet, filters: Option[JoinGraphInfo]): Option[JoinPlan] = { @@ -329,7 +329,7 @@ object JoinReorderDP extends PredicateHelper with Logging { case class JoinPlan( itemIds: Set[Int], plan: LogicalPlan, - joinConds: Set[Expression], + joinConds: ExpressionSet, planCost: Cost) { /** Get the cost of the root node of this plan tree. */ @@ -387,7 +387,7 @@ object JoinReorderDPFilters extends PredicateHelper { def buildJoinGraphInfo( conf: SQLConf, items: Seq[LogicalPlan], - conditions: Set[Expression], + conditions: ExpressionSet, itemIndex: Seq[(LogicalPlan, Int)]): Option[JoinGraphInfo] = { if (conf.joinReorderDPStarFilter) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index bcdc5cd942e35..296fe86e834e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -921,13 +921,13 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] private def getAllConstraints( left: LogicalPlan, right: LogicalPlan, - conditionOpt: Option[Expression]): Set[Expression] = { + conditionOpt: Option[Expression]): ExpressionSet = { val baseConstraints = left.constraints.union(right.constraints) - .union(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil).toSet) + .union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil))) baseConstraints.union(inferAdditionalConstraints(baseConstraints)) } - private def inferNewFilter(plan: LogicalPlan, constraints: Set[Expression]): LogicalPlan = { + private def inferNewFilter(plan: LogicalPlan, constraints: ExpressionSet): LogicalPlan = { val newPredicates = constraints .union(constructIsNotNullConstraints(constraints, plan.output)) .filter { c => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ec9bf90247f88..96c550616065a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -169,8 +169,8 @@ abstract class UnaryNode extends LogicalPlan { * Generates all valid constraints including an set of aliased constraints by replacing the * original constraint expressions with the corresponding alias */ - protected def getAllValidConstraints(projectList: Seq[NamedExpression]): Set[Expression] = { - var allConstraints = child.constraints.asInstanceOf[Set[Expression]] + protected def getAllValidConstraints(projectList: Seq[NamedExpression]): ExpressionSet = { + var allConstraints = child.constraints projectList.foreach { case a @ Alias(l: Literal, _) => allConstraints += EqualNullSafe(a.toAttribute, l) @@ -187,7 +187,7 @@ abstract class UnaryNode extends LogicalPlan { allConstraints } - override protected lazy val validConstraints: Set[Expression] = child.constraints + override protected lazy val validConstraints: ExpressionSet = child.constraints } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index 4c4ec000d0930..c4243da7b9e4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -29,16 +29,14 @@ trait QueryPlanConstraints extends ConstraintHelper { self: LogicalPlan => */ lazy val constraints: ExpressionSet = { if (conf.constraintPropagationEnabled) { - ExpressionSet( - validConstraints - .union(inferAdditionalConstraints(validConstraints)) - .union(constructIsNotNullConstraints(validConstraints, output)) - .filter { c => - c.references.nonEmpty && c.references.subsetOf(outputSet) && c.deterministic - } - ) + validConstraints + .union(inferAdditionalConstraints(validConstraints)) + .union(constructIsNotNullConstraints(validConstraints, output)) + .filter { c => + c.references.nonEmpty && c.references.subsetOf(outputSet) && c.deterministic + } } else { - ExpressionSet(Set.empty) + ExpressionSet() } } @@ -50,7 +48,7 @@ trait QueryPlanConstraints extends ConstraintHelper { self: LogicalPlan => * * See [[Canonicalize]] for more details. */ - protected lazy val validConstraints: Set[Expression] = Set.empty + protected lazy val validConstraints: ExpressionSet = ExpressionSet() } trait ConstraintHelper { @@ -60,8 +58,8 @@ trait ConstraintHelper { * For e.g., if an operator has constraints of the form (`a = 5`, `a = b`), this returns an * additional constraint of the form `b = 5`. */ - def inferAdditionalConstraints(constraints: Set[Expression]): Set[Expression] = { - var inferredConstraints = Set.empty[Expression] + def inferAdditionalConstraints(constraints: ExpressionSet): ExpressionSet = { + var inferredConstraints = ExpressionSet() // IsNotNull should be constructed by `constructIsNotNullConstraints`. val predicates = constraints.filterNot(_.isInstanceOf[IsNotNull]) predicates.foreach { @@ -79,9 +77,9 @@ trait ConstraintHelper { } private def replaceConstraints( - constraints: Set[Expression], + constraints: ExpressionSet, source: Expression, - destination: Expression): Set[Expression] = constraints.map(_ transform { + destination: Expression): ExpressionSet = constraints.map(_ transform { case e: Expression if e.semanticEquals(source) => destination }) @@ -91,15 +89,15 @@ trait ConstraintHelper { * returns a constraint of the form `isNotNull(a)` */ def constructIsNotNullConstraints( - constraints: Set[Expression], - output: Seq[Attribute]): Set[Expression] = { + constraints: ExpressionSet, + output: Seq[Attribute]): ExpressionSet = { // First, we propagate constraints from the null intolerant expressions. - var isNotNullConstraints: Set[Expression] = constraints.flatMap(inferIsNotNullConstraints) + var isNotNullConstraints = constraints.flatMap(inferIsNotNullConstraints(_)) // Second, we infer additional constraints from non-nullable attributes that are part of the // operator's output val nonNullableAttributes = output.filterNot(_.nullable) - isNotNullConstraints ++= nonNullableAttributes.map(IsNotNull).toSet + isNotNullConstraints ++= nonNullableAttributes.map(IsNotNull) isNotNullConstraints -- constraints } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index c83f6a376aa8c..223ef652d2f80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -75,7 +75,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } - override lazy val validConstraints: Set[Expression] = + override lazy val validConstraints: ExpressionSet = getAllValidConstraints(projectList) } @@ -143,10 +143,10 @@ case class Filter(condition: Expression, child: LogicalPlan) override def maxRows: Option[Long] = child.maxRows - override protected lazy val validConstraints: Set[Expression] = { + override protected lazy val validConstraints: ExpressionSet = { val predicates = splitConjunctivePredicates(condition) .filterNot(SubqueryExpression.hasCorrelatedSubquery) - child.constraints.union(predicates.toSet) + child.constraints.union(ExpressionSet(predicates)) } } @@ -154,9 +154,9 @@ abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends Binar def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty - protected def leftConstraints: Set[Expression] = left.constraints + protected def leftConstraints: ExpressionSet = left.constraints - protected def rightConstraints: Set[Expression] = { + protected def rightConstraints: ExpressionSet = { require(left.output.size == right.output.size) val attributeRewrites = AttributeMap(right.output.zip(left.output)) right.constraints.map(_ transform { @@ -188,7 +188,7 @@ case class Intersect( leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) } - override protected lazy val validConstraints: Set[Expression] = + override protected lazy val validConstraints: ExpressionSet = leftConstraints.union(rightConstraints) override def maxRows: Option[Long] = { @@ -208,7 +208,7 @@ case class Except( /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output - override protected lazy val validConstraints: Set[Expression] = leftConstraints + override protected lazy val validConstraints: ExpressionSet = leftConstraints } /** Factory for constructing new `Union` nodes. */ @@ -292,7 +292,7 @@ case class Union( private def rewriteConstraints( reference: Seq[Attribute], original: Seq[Attribute], - constraints: Set[Expression]): Set[Expression] = { + constraints: ExpressionSet): ExpressionSet = { require(reference.size == original.size) val attributeRewrites = AttributeMap(original.zip(reference)) constraints.map(_ transform { @@ -300,7 +300,7 @@ case class Union( }) } - private def merge(a: Set[Expression], b: Set[Expression]): Set[Expression] = { + private def merge(a: ExpressionSet, b: ExpressionSet): ExpressionSet = { val common = a.intersect(b) // The constraint with only one reference could be easily inferred as predicate // Grouping the constraints by it's references so we can combine the constraints with same @@ -314,7 +314,7 @@ case class Union( common ++ others } - override protected lazy val validConstraints: Set[Expression] = { + override protected lazy val validConstraints: ExpressionSet = { children .map(child => rewriteConstraints(children.head.output, child.output, child.constraints)) .reduce(merge(_, _)) @@ -346,15 +346,15 @@ case class Join( } } - override protected lazy val validConstraints: Set[Expression] = { + override protected lazy val validConstraints: ExpressionSet = { joinType match { case _: InnerLike if condition.isDefined => left.constraints .union(right.constraints) - .union(splitConjunctivePredicates(condition.get).toSet) + .union(ExpressionSet(splitConjunctivePredicates(condition.get))) case LeftSemi if condition.isDefined => left.constraints - .union(splitConjunctivePredicates(condition.get).toSet) + .union(ExpressionSet(splitConjunctivePredicates(condition.get))) case j: ExistenceJoin => left.constraints case _: InnerLike => @@ -366,7 +366,7 @@ case class Join( case RightOuter => right.constraints case FullOuter => - Set.empty[Expression] + ExpressionSet() } } @@ -588,7 +588,7 @@ case class Aggregate( override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) override def maxRows: Option[Long] = child.maxRows - override lazy val validConstraints: Set[Expression] = { + override lazy val validConstraints: ExpressionSet = { val nonAgg = aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) getAllValidConstraints(nonAgg) } @@ -717,7 +717,7 @@ case class Expand( // This operator can reuse attributes (for example making them null when doing a roll up) so // the constraints of the child may no longer be valid. - override protected lazy val validConstraints: Set[Expression] = Set.empty[Expression] + override protected lazy val validConstraints: ExpressionSet = ExpressionSet() } /** diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 4caad37094cd6..d39916159cb37 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -93,7 +93,7 @@ Condition : (isnotnull(ws_sold_date_sk#5) AND isnotnull(ws_bill_customer_sk#6)) Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 3] @@ -101,7 +101,7 @@ Input [3]: [d_date_sk#7, d_year#8, d_moy#9] (11) Filter [codegen id : 3] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_moy#9) AND isnotnull(d_year#8)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) (12) Project [codegen id : 3] Output [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt index 056eb273bfe50..7d5fa795f98f8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/simplified.txt @@ -1,18 +1,18 @@ -TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] +TakeOrderedAndProject [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,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (14) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count,count(1)] + HashAggregate [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] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter - Exchange [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [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] #1 WholeStageCodegen (13) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [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,count] + Project [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] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (12) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] InputAdapter SortMergeJoin [c_customer_sk,customer_sk] SortMergeJoin [c_customer_sk,customer_sk] @@ -21,10 +21,10 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep InputAdapter Exchange [c_customer_sk] #3 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (7) Sort [customer_sk] InputAdapter @@ -32,26 +32,26 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Union WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (6) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_ship_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (10) @@ -60,22 +60,22 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Exchange [customer_sk] #6 WholeStageCodegen (9) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #7 WholeStageCodegen (11) Project [ca_address_sk] - Filter [ca_address_sk,ca_county] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/simplified.txt index 41685e1f311a0..af1d5a891165b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/simplified.txt @@ -1,64 +1,64 @@ -TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] +TakeOrderedAndProject [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,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (10) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count,count(1)] + HashAggregate [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] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter - Exchange [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [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] #1 WholeStageCodegen (9) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [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,count] + Project [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] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,customer_sk] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 Union WholeStageCodegen (2) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (4) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_ship_customer_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_ship_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 WholeStageCodegen (6) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 WholeStageCodegen (7) Project [ca_address_sk] - Filter [ca_address_sk,ca_county] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] @@ -68,4 +68,4 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/simplified.txt index f0e796970f414..2343a1322b340 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/simplified.txt @@ -1,46 +1,46 @@ -TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] +TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (7) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter - Exchange [i_brand,i_brand_id,i_manufact,i_manufact_id] #1 + Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 WholeStageCodegen (6) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,s_zip] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) - Project [c_current_addr_sk,s_zip,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [c_current_addr_sk,ss_ext_sales_price,ss_item_sk,ss_store_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] - Filter [c_current_addr_sk,c_customer_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Filter [s_store_sk,s_zip] + Filter [s_zip,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_zip] @@ -51,8 +51,8 @@ TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Project [i_brand,i_brand_id,i_item_sk,i_manufact,i_manufact_id] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id,i_manufact,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/simplified.txt index 9217520556863..1bbbf35e4d510 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] +TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (7) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter - Exchange [i_brand,i_brand_id,i_manufact,i_manufact_id] #1 + Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 WholeStageCodegen (6) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] - BroadcastHashJoin [ca_zip,s_store_sk,s_zip,ss_store_sk] - Project [ca_zip,i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price,ss_store_sk] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_customer_sk,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_store_sk] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk,i_manufact,i_manufact_id] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id,i_manufact,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -52,7 +52,7 @@ TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [s_store_sk,s_zip] + Filter [s_zip,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt index 0c568bacff1d0..fa01042350149 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt @@ -82,7 +82,7 @@ TakeOrderedAndProject (77) Output [2]: [d_date_sk#1, d_year#2] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), LessThanOrEqual(d_date_sk,2451910), GreaterThanOrEqual(d_date_sk,2451545), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -90,7 +90,7 @@ Input [2]: [d_date_sk#1, d_year#2] (3) Filter [codegen id : 1] Input [2]: [d_date_sk#1, d_year#2] -Condition : ((((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND (d_date_sk#1 <= 2451910)) AND (d_date_sk#1 >= 2451545)) AND isnotnull(d_date_sk#1)) +Condition : ((((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND (d_date_sk#1 >= 2451545)) AND (d_date_sk#1 <= 2451910)) AND isnotnull(d_date_sk#1)) (4) Project [codegen id : 1] Output [1]: [d_date_sk#1] @@ -127,7 +127,7 @@ Input [9]: [d_date_sk#1, ss_sold_date_sk#4, ss_item_sk#5, ss_cdemo_sk#6, ss_stor Output [4]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_education_status), IsNotNull(cd_gender), IsNotNull(cd_marital_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,D), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,D), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] @@ -135,7 +135,7 @@ Input [4]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_stat (13) Filter [codegen id : 2] Input [4]: [cd_demo_sk#12, cd_gender#13, cd_marital_status#14, cd_education_status#15] -Condition : ((((((isnotnull(cd_education_status#15) AND isnotnull(cd_gender#13)) AND isnotnull(cd_marital_status#14)) AND (cd_gender#13 = F)) AND (cd_marital_status#14 = D)) AND (cd_education_status#15 = Primary)) AND isnotnull(cd_demo_sk#12)) +Condition : ((((((isnotnull(cd_gender#13) AND isnotnull(cd_marital_status#14)) AND isnotnull(cd_education_status#15)) AND (cd_gender#13 = F)) AND (cd_marital_status#14 = D)) AND (cd_education_status#15 = Primary)) AND isnotnull(cd_demo_sk#12)) (14) Project [codegen id : 2] Output [1]: [cd_demo_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt index f2d9cfb0b28af..fc7202e739bcc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/simplified.txt @@ -1,73 +1,73 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,s_state,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,sum,sum,sum,sum] + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state] #1 WholeStageCodegen (5) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id,s_state] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] WholeStageCodegen (12) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,s_state,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),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] InputAdapter Exchange [i_item_id] #6 WholeStageCodegen (11) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter ReusedExchange [d_date_sk] #2 - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -75,31 +75,31 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter ReusedExchange [cd_demo_sk] #3 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [count,count,count,count,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,i_item_id,s_state,sum,sum,sum,sum] + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange #8 WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter ReusedExchange [d_date_sk] #2 - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [s_store_sk] #7 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt index 6e0bd7a6c32ab..c9724bed3e593 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt @@ -96,7 +96,7 @@ Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545) Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_gender), EqualTo(cd_gender,F), EqualTo(cd_marital_status,D), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,D), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -104,7 +104,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_marital_status#11) AND isnotnull(cd_education_status#12)) AND isnotnull(cd_gender#10)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = D)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = D)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] @@ -127,7 +127,7 @@ Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_store_sk#4, ss_qu Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), LessThanOrEqual(d_date_sk,2451910), GreaterThanOrEqual(d_date_sk,2451545), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] @@ -135,7 +135,7 @@ Input [2]: [d_date_sk#14, d_year#15] (13) Filter [codegen id : 2] Input [2]: [d_date_sk#14, d_year#15] -Condition : ((((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND (d_date_sk#14 <= 2451910)) AND (d_date_sk#14 >= 2451545)) AND isnotnull(d_date_sk#14)) +Condition : ((((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND (d_date_sk#14 >= 2451545)) AND (d_date_sk#14 <= 2451910)) AND isnotnull(d_date_sk#14)) (14) Project [codegen id : 2] Output [1]: [d_date_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/simplified.txt index 7bfdbae974b51..7f4d66577e9c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,s_state,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,sum,sum,sum,sum] + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state] #1 WholeStageCodegen (5) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id,s_state] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -40,32 +40,32 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] WholeStageCodegen (12) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,s_state,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),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] InputAdapter Exchange [i_item_id] #6 WholeStageCodegen (11) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [cd_demo_sk] #2 InputAdapter @@ -77,27 +77,27 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [count,count,count,count,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,i_item_id,s_state,sum,sum,sum,sum] + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange #8 WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [cd_demo_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/simplified.txt index 733c9c780db94..6ab00e122c5c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,sum_agg] +TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,sum,sum(UnscaledValue(ss_net_profit)),sum_agg] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_net_profit)),brand_id,brand,sum_agg,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_net_profit] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_net_profit] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_brand_id,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [d_year,ss_net_profit,i_brand_id,i_brand] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_net_profit,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_profit,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy] + Filter [d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/simplified.txt index 0fc4dc7b931f4..56123afb1672c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,sum_agg] +TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,sum,sum(UnscaledValue(ss_net_profit)),sum_agg] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_net_profit)),brand_id,brand,sum_agg,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_net_profit] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_net_profit] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [d_year,ss_net_profit,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [d_year,ss_item_sk,ss_net_profit] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy] + Filter [d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_profit,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt index 94ec6ce18503e..d9b416ddba9ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (10) - Sort [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter - Exchange [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] #1 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 WholeStageCodegen (9) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] @@ -12,29 +12,29 @@ WholeStageCodegen (10) Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_customer_sk] #3 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -42,15 +42,15 @@ WholeStageCodegen (10) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] @@ -60,4 +60,4 @@ WholeStageCodegen (10) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt index 12cd87e119622..5af07f1d4ddef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/simplified.txt @@ -1,34 +1,34 @@ WholeStageCodegen (7) - Sort [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter - Exchange [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] #1 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 WholeStageCodegen (6) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk] #2 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,19 +36,19 @@ WholeStageCodegen (7) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt index 3feb5c7308cf5..10f7d4b9d3b4b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject (21) Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2451149), IsNotNull(d_date_sk), LessThanOrEqual(d_date_sk,2451179)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2451149), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -34,7 +34,7 @@ Input [3]: [d_date_sk#1, d_year#2, d_moy#3] (3) Filter [codegen id : 1] Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2451149)) AND isnotnull(d_date_sk#1)) AND (d_date_sk#1 <= 2451179)) +Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2451149)) AND (d_date_sk#1 <= 2451179)) AND isnotnull(d_date_sk#1)) (4) Project [codegen id : 1] Output [2]: [d_date_sk#1, d_year#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/simplified.txt index 4be03d5b3ed7d..96e2f0b12cfb1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [d_year,i_category,i_category_id,sum(ss_ext_sales_price)] +TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (4) - HashAggregate [d_year,i_category,i_category_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price)] + HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] InputAdapter - Exchange [d_year,i_category,i_category_id] #1 + Exchange [d_year,i_category_id,i_category] #1 WholeStageCodegen (3) - HashAggregate [d_year,i_category,i_category_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_category,i_category_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [ss_item_sk,ss_sold_date_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_category,i_category_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_category_id,i_category] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_category_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt index 86ab1688b8243..e8e93409817b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject (21) Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), LessThanOrEqual(d_date_sk,2451179), GreaterThanOrEqual(d_date_sk,2451149), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2451149), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 3] @@ -34,7 +34,7 @@ Input [3]: [d_date_sk#1, d_year#2, d_moy#3] (3) Filter [codegen id : 3] Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 <= 2451179)) AND (d_date_sk#1 >= 2451149)) AND isnotnull(d_date_sk#1)) +Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2451149)) AND (d_date_sk#1 <= 2451179)) AND isnotnull(d_date_sk#1)) (4) Project [codegen id : 3] Output [2]: [d_date_sk#1, d_year#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/simplified.txt index c8fc6e5c63919..d9bb6de204184 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [d_year,i_category,i_category_id,sum(ss_ext_sales_price)] +TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (4) - HashAggregate [d_year,i_category,i_category_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price)] + HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] InputAdapter - Exchange [d_year,i_category,i_category_id] #1 + Exchange [d_year,i_category_id,i_category] #1 WholeStageCodegen (3) - HashAggregate [d_year,i_category,i_category_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_category,i_category_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_category,i_category_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_category_id,i_category] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_category_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/simplified.txt index 2ed0f8f445720..a366ea3d4696a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] +TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] WholeStageCodegen (4) - HashAggregate [s_store_id,s_store_name,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter - Exchange [s_store_id,s_store_name] #1 + Exchange [s_store_name,s_store_id] #1 WholeStageCodegen (3) - HashAggregate [d_day_name,s_store_id,s_store_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,s_store_id,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_day_name,ss_sales_price,ss_store_sk] + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk,d_day_name] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_day_name] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [s_store_id,s_store_name,s_store_sk] + Project [s_store_sk,s_store_id,s_store_name] Filter [s_gmt_offset,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_gmt_offset,s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/simplified.txt index bf8ac2624db70..d2cb50bd5a41d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] +TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] WholeStageCodegen (4) - HashAggregate [s_store_id,s_store_name,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter - Exchange [s_store_id,s_store_name] #1 + Exchange [s_store_name,s_store_id] #1 WholeStageCodegen (3) - HashAggregate [d_day_name,s_store_id,s_store_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,s_store_id,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_day_name,ss_sales_price,ss_store_sk] + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_day_name] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_day_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [s_store_id,s_store_name,s_store_sk] + Project [s_store_sk,s_store_id,s_store_name] Filter [s_gmt_offset,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_gmt_offset,s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt index 8108d64db305b..bc83e3b7e2eba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/simplified.txt @@ -1,49 +1,49 @@ -TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] +TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] WholeStageCodegen (16) - Project [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] - SortMergeJoin [bought_city,c_current_addr_sk,ca_address_sk,ca_city] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + SortMergeJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] InputAdapter WholeStageCodegen (13) Sort [c_current_addr_sk] InputAdapter Exchange [c_current_addr_sk] #1 WholeStageCodegen (12) - Project [amt,bought_city,c_current_addr_sk,c_first_name,c_last_name,profit,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (9) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #2 WholeStageCodegen (8) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,bought_city,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - HashAggregate [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - SortMergeJoin [ca_address_sk,ss_addr_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (5) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #3 WholeStageCodegen (4) - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -51,12 +51,12 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -76,10 +76,10 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s InputAdapter Exchange [c_customer_sk] #8 WholeStageCodegen (10) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (15) Sort [ca_address_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/simplified.txt index edf9894f202ea..abdc7a3babccd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] +TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] WholeStageCodegen (8) - Project [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] - BroadcastHashJoin [bought_city,c_current_addr_sk,ca_address_sk,ca_city] - Project [amt,bought_city,c_current_addr_sk,c_first_name,c_last_name,profit,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,bought_city,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] + Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] InputAdapter - Exchange [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #1 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 WholeStageCodegen (5) - HashAggregate [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -36,12 +36,12 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -55,9 +55,9 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s InputAdapter BroadcastExchange #6 WholeStageCodegen (6) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt index bb606f6860124..aa33cd1be7482 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject (21) Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2451149), IsNotNull(d_date_sk), LessThanOrEqual(d_date_sk,2451179)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2451149), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -34,7 +34,7 @@ Input [3]: [d_date_sk#1, d_year#2, d_moy#3] (3) Filter [codegen id : 1] Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2451149)) AND isnotnull(d_date_sk#1)) AND (d_date_sk#1 <= 2451179)) +Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2451149)) AND (d_date_sk#1 <= 2451179)) AND isnotnull(d_date_sk#1)) (4) Project [codegen id : 1] Output [2]: [d_date_sk#1, d_year#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/simplified.txt index 290d6f35ea78c..f4aaf3df75135 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,ext_price] +TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [ss_item_sk,ss_sold_date_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt index bc7bc63475fd2..42944d42ed30c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt @@ -26,7 +26,7 @@ TakeOrderedAndProject (21) Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), LessThanOrEqual(d_date_sk,2451179), GreaterThanOrEqual(d_date_sk,2451149), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2451149), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 3] @@ -34,7 +34,7 @@ Input [3]: [d_date_sk#1, d_year#2, d_moy#3] (3) Filter [codegen id : 3] Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 <= 2451179)) AND (d_date_sk#1 >= 2451149)) AND isnotnull(d_date_sk#1)) +Condition : ((((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 12)) AND (d_year#2 = 1998)) AND (d_date_sk#1 >= 2451149)) AND (d_date_sk#1 <= 2451179)) AND isnotnull(d_date_sk#1)) (4) Project [codegen id : 3] Output [2]: [d_date_sk#1, d_year#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/simplified.txt index 7d70658c38241..ba48ad5b699f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,ext_price] +TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index 237ad5594b860..10db0b7d4d460 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -109,7 +109,7 @@ Input [5]: [i_manufact_id#5, ss_sold_date_sk#11, ss_store_sk#13, ss_sales_price# Output [3]: [d_date_sk#17, d_month_seq#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1222,1215,1223,1217,1214,1219,1213,1218,1220,1221,1216,1212]), LessThanOrEqual(d_date_sk,2452275), GreaterThanOrEqual(d_date_sk,2451911), IsNotNull(d_date_sk)] +PushedFilters: [In(d_month_seq, [1222,1215,1223,1217,1214,1219,1213,1218,1220,1221,1216,1212]), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [3]: [d_date_sk#17, d_month_seq#18, d_qoy#19] (19) Filter [codegen id : 3] Input [3]: [d_date_sk#17, d_month_seq#18, d_qoy#19] -Condition : (((d_month_seq#18 INSET (1222,1215,1223,1217,1214,1219,1213,1218,1220,1221,1216,1212) AND (d_date_sk#17 <= 2452275)) AND (d_date_sk#17 >= 2451911)) AND isnotnull(d_date_sk#17)) +Condition : (((d_month_seq#18 INSET (1222,1215,1223,1217,1214,1219,1213,1218,1220,1221,1216,1212) AND (d_date_sk#17 >= 2451911)) AND (d_date_sk#17 <= 2452275)) AND isnotnull(d_date_sk#17)) (20) Project [codegen id : 3] Output [2]: [d_date_sk#17, d_qoy#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/simplified.txt index 98f126325517b..6b3f9b5ab3eff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (7) - Project [avg_quarterly_sales,i_manufact_id,sum_sales] + Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] @@ -9,29 +9,29 @@ TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] InputAdapter Exchange [i_manufact_id] #1 WholeStageCodegen (5) - HashAggregate [d_qoy,i_manufact_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_qoy,i_manufact_id] #2 + Exchange [i_manufact_id,d_qoy] #2 WholeStageCodegen (4) - HashAggregate [d_qoy,i_manufact_id,ss_sales_price] [sum,sum] - Project [d_qoy,i_manufact_id,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manufact_id,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_manufact_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_sold_date_sk,ss_sales_price] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [i_item_sk,i_manufact_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manufact_id] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -43,7 +43,7 @@ TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk,d_qoy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_qoy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/simplified.txt index c8070c46c9c80..0e01f08343acd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (7) - Project [avg_quarterly_sales,i_manufact_id,sum_sales] + Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] @@ -9,34 +9,34 @@ TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] InputAdapter Exchange [i_manufact_id] #1 WholeStageCodegen (5) - HashAggregate [d_qoy,i_manufact_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_qoy,i_manufact_id] #2 + Exchange [i_manufact_id,d_qoy] #2 WholeStageCodegen (4) - HashAggregate [d_qoy,i_manufact_id,ss_sales_price] [sum,sum] - Project [d_qoy,i_manufact_id,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_qoy,i_manufact_id,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manufact_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] Project [i_item_sk,i_manufact_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_qoy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_qoy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/simplified.txt index dab4e1d1bd68b..4f375c80678e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,ext_price] +TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [ss_item_sk,ss_sold_date_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/simplified.txt index b20fb74817e84..bb2ef8e82515c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,ext_price] +TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt index 0650ed03a2e8c..1ea8b4892596d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_sales1 / sat_sales2),(sun_sales1 / sun_sales2),(thu_sales1 / thu_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),d_week_seq1,s_store_id1,s_store_name1] +TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) - Project [d_week_seq1,fri_sales1,fri_sales2,mon_sales1,mon_sales2,s_store_id1,s_store_name1,sat_sales1,sat_sales2,sun_sales1,sun_sales2,thu_sales1,thu_sales2,tue_sales1,wed_sales1,wed_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2,s_store_id1,s_store_id2] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #1 WholeStageCodegen (2) - HashAggregate [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -42,30 +42,30 @@ TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_ InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,wed_sales] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #6 WholeStageCodegen (6) - HashAggregate [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_day_name,d_week_seq] #2 + ReusedExchange [d_date_sk,d_week_seq,d_day_name] #2 InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt index 0650ed03a2e8c..1ea8b4892596d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_sales1 / sat_sales2),(sun_sales1 / sun_sales2),(thu_sales1 / thu_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),d_week_seq1,s_store_id1,s_store_name1] +TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales1),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) - Project [d_week_seq1,fri_sales1,fri_sales2,mon_sales1,mon_sales2,s_store_id1,s_store_name1,sat_sales1,sat_sales2,sun_sales1,sun_sales2,thu_sales1,thu_sales2,tue_sales1,wed_sales1,wed_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2,s_store_id1,s_store_id2] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #1 WholeStageCodegen (2) - HashAggregate [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -42,30 +42,30 @@ TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_ InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,wed_sales] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #6 WholeStageCodegen (6) - HashAggregate [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_day_name,d_week_seq] #2 + ReusedExchange [d_date_sk,d_week_seq,d_day_name] #2 InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index 543c374aa784f..5c4a69d38637d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -109,7 +109,7 @@ Input [5]: [i_manager_id#5, ss_sold_date_sk#11, ss_store_sk#13, ss_sales_price#1 Output [3]: [d_date_sk#17, d_month_seq#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229]), LessThanOrEqual(d_date_sk,2452487), GreaterThanOrEqual(d_date_sk,2452123), IsNotNull(d_date_sk)] +PushedFilters: [In(d_month_seq, [1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229]), GreaterThanOrEqual(d_date_sk,2452123), LessThanOrEqual(d_date_sk,2452487), IsNotNull(d_date_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [3]: [d_date_sk#17, d_month_seq#18, d_moy#19] (19) Filter [codegen id : 3] Input [3]: [d_date_sk#17, d_month_seq#18, d_moy#19] -Condition : (((d_month_seq#18 INSET (1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229) AND (d_date_sk#17 <= 2452487)) AND (d_date_sk#17 >= 2452123)) AND isnotnull(d_date_sk#17)) +Condition : (((d_month_seq#18 INSET (1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229) AND (d_date_sk#17 >= 2452123)) AND (d_date_sk#17 <= 2452487)) AND isnotnull(d_date_sk#17)) (20) Project [codegen id : 3] Output [2]: [d_date_sk#17, d_moy#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/simplified.txt index 99d321b52694a..3570e17b2f176 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (7) - Project [avg_monthly_sales,i_manager_id,sum_sales] + Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] @@ -9,29 +9,29 @@ TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] InputAdapter Exchange [i_manager_id] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_manager_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_manager_id] #2 + Exchange [i_manager_id,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_manager_id,ss_sales_price] [sum,sum] - Project [d_moy,i_manager_id,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manager_id,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_manager_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_sold_date_sk,ss_sales_price] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [i_item_sk,i_manager_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manager_id] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -43,7 +43,7 @@ TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index f8faa8cc6a4a6..c0d107d6ae851 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -82,7 +82,7 @@ Input [6]: [i_item_sk#1, i_manager_id#5, ss_sold_date_sk#10, ss_item_sk#11, ss_s Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229]), LessThanOrEqual(d_date_sk,2452487), GreaterThanOrEqual(d_date_sk,2452123), IsNotNull(d_date_sk)] +PushedFilters: [In(d_month_seq, [1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229]), GreaterThanOrEqual(d_date_sk,2452123), LessThanOrEqual(d_date_sk,2452487), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] @@ -90,7 +90,7 @@ Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] (13) Filter [codegen id : 2] Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (((d_month_seq#16 INSET (1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229) AND (d_date_sk#15 <= 2452487)) AND (d_date_sk#15 >= 2452123)) AND isnotnull(d_date_sk#15)) +Condition : (((d_month_seq#16 INSET (1222,1228,1223,1227,1219,1226,1224,1225,1230,1220,1221,1229) AND (d_date_sk#15 >= 2452123)) AND (d_date_sk#15 <= 2452487)) AND isnotnull(d_date_sk#15)) (14) Project [codegen id : 2] Output [2]: [d_date_sk#15, d_moy#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/simplified.txt index 7d1b1e0ae9c41..d9dae62e25543 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (7) - Project [avg_monthly_sales,i_manager_id,sum_sales] + Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] @@ -9,34 +9,34 @@ TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] InputAdapter Exchange [i_manager_id] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_manager_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_manager_id] #2 + Exchange [i_manager_id,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_manager_id,ss_sales_price] [sum,sum] - Project [d_moy,i_manager_id,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,i_manager_id,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manager_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] Project [i_item_sk,i_manager_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt index 872ce5199073f..21aa4974877f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt @@ -1,63 +1,63 @@ -TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] +TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) - Project [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] InputAdapter BroadcastExchange #1 WholeStageCodegen (8) - Project [revenue,s_store_name,ss_item_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + Project [ss_item_sk,revenue,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (7) - Project [revenue,ss_item_sk,ss_store_sk] - BroadcastHashJoin [ave,revenue,ss_store_sk,ss_store_sk] + Project [ss_store_sk,ss_item_sk,revenue] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Filter [revenue] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #3 + Exchange [ss_store_sk,ss_item_sk] #3 WholeStageCodegen (2) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #5 WholeStageCodegen (6) - HashAggregate [count,ss_store_sk,sum] [ave,avg(revenue),count,sum] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (5) - HashAggregate [revenue,ss_store_sk] [count,count,sum,sum] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #7 + Exchange [ss_store_sk,ss_item_sk] #7 WholeStageCodegen (4) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter ReusedExchange [d_date_sk] #4 Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_current_price,i_item_desc,i_item_sk,i_wholesale_cost] + Scan parquet default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt index 77f6cc15c8210..a4b468ffeb0c9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] +TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) - Project [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] - BroadcastHashJoin [ave,revenue,ss_store_sk,ss_store_sk] - Project [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [revenue,s_store_name,ss_item_sk,ss_store_sk] + Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [s_store_name,ss_store_sk,ss_item_sk,revenue] BroadcastHashJoin [s_store_sk,ss_store_sk] Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 WholeStageCodegen (3) Filter [revenue] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #2 + Exchange [ss_store_sk,ss_item_sk] #2 WholeStageCodegen (2) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -39,25 +39,25 @@ TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,reve Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_current_price,i_item_desc,i_item_sk,i_wholesale_cost] + Scan parquet default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - HashAggregate [count,ss_store_sk,sum] [ave,avg(revenue),count,sum] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (7) - HashAggregate [revenue,ss_store_sk] [count,count,sum,sum] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #7 + Exchange [ss_store_sk,ss_item_sk] #7 WholeStageCodegen (6) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/simplified.txt index 510bcd475291d..6fd7148b9a8fb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/simplified.txt @@ -1,49 +1,49 @@ -TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] +TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] WholeStageCodegen (14) - Project [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] - SortMergeJoin [bought_city,c_current_addr_sk,ca_address_sk,ca_city] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + SortMergeJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] InputAdapter WholeStageCodegen (11) Sort [c_current_addr_sk] InputAdapter Exchange [c_current_addr_sk] #1 WholeStageCodegen (10) - Project [bought_city,c_current_addr_sk,c_first_name,c_last_name,extended_price,extended_tax,list_price,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #2 WholeStageCodegen (6) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum,sum] [bought_city,extended_price,extended_tax,list_price,sum,sum,sum,sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_tax))] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] InputAdapter - Exchange [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #3 WholeStageCodegen (5) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] [sum,sum,sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -51,12 +51,12 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #7 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -70,10 +70,10 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri InputAdapter Exchange [c_customer_sk] #8 WholeStageCodegen (8) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (13) Sort [ca_address_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/simplified.txt index 4c2d24b06c709..819e5504ff673 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] +TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] WholeStageCodegen (8) - Project [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] - BroadcastHashJoin [bought_city,c_current_addr_sk,ca_address_sk,ca_city] - Project [bought_city,c_current_addr_sk,c_first_name,c_last_name,extended_price,extended_tax,list_price,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum,sum] [bought_city,extended_price,extended_tax,list_price,sum,sum,sum,sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_tax))] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] + Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] InputAdapter - Exchange [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #1 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 WholeStageCodegen (5) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] [sum,sum,sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -36,12 +36,12 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -55,9 +55,9 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri InputAdapter BroadcastExchange #6 WholeStageCodegen (6) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index b20ba979588d9..6071139e809cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -115,7 +115,7 @@ Input [8]: [ss_item_sk#5, ss_cdemo_sk#6, ss_promo_sk#7, ss_quantity#8, ss_list_p Output [4]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_education_status), IsNotNull(cd_gender), IsNotNull(cd_marital_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -123,7 +123,7 @@ Input [4]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_stat (20) Filter [codegen id : 3] Input [4]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19] -Condition : ((((((isnotnull(cd_education_status#19) AND isnotnull(cd_gender#17)) AND isnotnull(cd_marital_status#18)) AND (cd_gender#17 = F)) AND (cd_marital_status#18 = W)) AND (cd_education_status#19 = Primary)) AND isnotnull(cd_demo_sk#16)) +Condition : ((((((isnotnull(cd_gender#17) AND isnotnull(cd_marital_status#18)) AND isnotnull(cd_education_status#19)) AND (cd_gender#17 = F)) AND (cd_marital_status#18 = W)) AND (cd_education_status#19 = Primary)) AND isnotnull(cd_demo_sk#16)) (21) Project [codegen id : 3] Output [1]: [cd_demo_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt index 8f283a7b8e59e..4576b8cef59ee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_sales_price)),avg(cast(ss_quantity as bigint)),count,count,count,count,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) - HashAggregate [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price] + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -32,19 +32,19 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [p_channel_email,p_channel_event,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_email,p_channel_event,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt index b8afa28a5f796..7cbeb89559367 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt @@ -53,7 +53,7 @@ Condition : (((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2450815) Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_gender), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_marital_status#11) AND isnotnull(cd_education_status#12)) AND isnotnull(cd_gender#10)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = W)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = W)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] @@ -84,7 +84,7 @@ Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_cdemo_sk#3, ss_promo_sk#4, ss_qu Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), LessThanOrEqual(d_date_sk,2451179), GreaterThanOrEqual(d_date_sk,2450815), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), GreaterThanOrEqual(d_date_sk,2450815), LessThanOrEqual(d_date_sk,2451179), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#14, d_year#15] (13) Filter [codegen id : 2] Input [2]: [d_date_sk#14, d_year#15] -Condition : ((((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND (d_date_sk#14 <= 2451179)) AND (d_date_sk#14 >= 2450815)) AND isnotnull(d_date_sk#14)) +Condition : ((((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND (d_date_sk#14 >= 2450815)) AND (d_date_sk#14 <= 2451179)) AND isnotnull(d_date_sk#14)) (14) Project [codegen id : 2] Output [1]: [d_date_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/simplified.txt index db56467a0218d..2d3ed4d1f7697 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_sales_price)),avg(cast(ss_quantity as bigint)),count,count,count,count,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) - HashAggregate [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk] + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_sold_date_sk,ss_cdemo_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -39,7 +39,7 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -47,4 +47,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [p_channel_email,p_channel_event,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_email,p_channel_event,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt index 4f19c79039220..7496388d3430c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/simplified.txt @@ -3,35 +3,35 @@ WholeStageCodegen (7) InputAdapter Exchange [cnt] #1 WholeStageCodegen (6) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (5) Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_customer_sk] #3 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -39,16 +39,16 @@ WholeStageCodegen (7) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt index 55312b6569a21..46b7241565719 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/simplified.txt @@ -3,32 +3,32 @@ WholeStageCodegen (7) InputAdapter Exchange [cnt] #1 WholeStageCodegen (6) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk] #2 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,19 +36,19 @@ WholeStageCodegen (7) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/simplified.txt index 8ef698c9f896c..e031b8c4c836c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/simplified.txt @@ -1,52 +1,52 @@ -TakeOrderedAndProject [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number,substr(s_city, 1, 30)] +TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] WholeStageCodegen (9) - Project [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #1 WholeStageCodegen (5) - HashAggregate [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] InputAdapter - Exchange [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #2 WholeStageCodegen (4) - HashAggregate [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Project [s_city,s_store_sk] + Project [s_store_sk,s_city] Filter [s_number_employees,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_number_employees,s_store_sk] + Scan parquet default.store [s_store_sk,s_number_employees,s_city] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/simplified.txt index 53f5a7d0acc0e..5b6177488766e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/simplified.txt @@ -1,43 +1,43 @@ -TakeOrderedAndProject [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number,substr(s_city, 1, 30)] +TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] WholeStageCodegen (6) - Project [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] InputAdapter - Exchange [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #1 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 WholeStageCodegen (4) - HashAggregate [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [s_city,s_store_sk] + Project [s_store_sk,s_city] Filter [s_number_employees,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_number_employees,s_store_sk] + Scan parquet default.store [s_store_sk,s_number_employees,s_city] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index a338144d26446..776d1262c7ffd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -50,7 +50,7 @@ Condition : ((((isnotnull(ss_sold_date_sk#1) AND (ss_sold_date_sk#1 >= 2451545)) Output [3]: [d_date_sk#5, d_year#6, d_moy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), LessThanOrEqual(d_date_sk,2451910), GreaterThanOrEqual(d_date_sk,2451545), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -58,7 +58,7 @@ Input [3]: [d_date_sk#5, d_year#6, d_moy#7] (6) Filter [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -Condition : ((((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND (d_date_sk#5 <= 2451910)) AND (d_date_sk#5 >= 2451545)) AND isnotnull(d_date_sk#5)) +Condition : ((((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND (d_date_sk#5 >= 2451545)) AND (d_date_sk#5 <= 2451910)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [2]: [d_date_sk#5, d_moy#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/simplified.txt index 6f7fbff7c58d8..500085fe982ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] WholeStageCodegen (7) - Project [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [_w0,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - Sort [i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name] #2 + Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_moy,s_company_name,s_store_name,ss_item_sk,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/simplified.txt index 3b3ad9cf61293..05b5cf036a3e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] WholeStageCodegen (7) - Project [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [_w0,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - Sort [i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name] #2 + Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,i_brand,i_category,i_class,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_category,i_class,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/simplified.txt index 8e55b010434aa..7146d246053b3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (10) - Project [i_category,i_class,i_current_price,i_item_desc,itemrevenue,revenueratio] - Sort [i_category,i_class,i_item_desc,i_item_id,revenueratio] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - Exchange [i_category,i_class,i_item_desc,i_item_id,revenueratio] #1 + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -12,25 +12,25 @@ WholeStageCodegen (10) InputAdapter Exchange [i_class] #2 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #3 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (6) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (2) - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -38,7 +38,7 @@ WholeStageCodegen (10) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -48,4 +48,4 @@ WholeStageCodegen (10) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/simplified.txt index 6f39176d9a9e1..0db6566a361c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (7) - Project [i_category,i_class,i_current_price,i_item_desc,itemrevenue,revenueratio] - Sort [i_category,i_class,i_item_desc,i_item_id,revenueratio] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - Exchange [i_category,i_class,i_item_desc,i_item_id,revenueratio] #1 + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -12,26 +12,26 @@ WholeStageCodegen (7) InputAdapter Exchange [i_class] #2 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #3 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (3) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -39,4 +39,4 @@ WholeStageCodegen (7) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max.sf100/simplified.txt index 31622e85a4a03..9883e30a3e4b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max.sf100/simplified.txt @@ -1,14 +1,14 @@ WholeStageCodegen (3) - HashAggregate [count,count,count,max,max,max,max,max,max,max,max,max] [count,count,count,count(1),count(ss_sold_date_sk),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk),max_ss_addr_sk,max_ss_cdemo_sk,max_ss_customer_sk,max_ss_hdemo_sk,max_ss_item_sk,max_ss_promo_sk,max_ss_sold_date_sk,max_ss_sold_time_sk,max_ss_store_sk,not_null_total,total,unique_days] + HashAggregate [count,count,max,max,max,max,max,max,max,max,max,count] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count(ss_sold_date_sk),total,not_null_total,unique_days,max_ss_sold_date_sk,max_ss_sold_time_sk,max_ss_item_sk,max_ss_customer_sk,max_ss_cdemo_sk,max_ss_hdemo_sk,max_ss_addr_sk,max_ss_store_sk,max_ss_promo_sk,count,count,max,max,max,max,max,max,max,max,max,count] InputAdapter Exchange #1 WholeStageCodegen (2) - HashAggregate [ss_sold_date_sk] [count,count,count,count,count,count,count(1),count(ss_sold_date_sk),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk)] - HashAggregate [ss_sold_date_sk] [count,count,count,count,count(1),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk)] + HashAggregate [ss_sold_date_sk] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count(ss_sold_date_sk),count,count,max,max,max,max,max,max,max,max,max,count,count,count,max,max,max,max,max,max,max,max,max,count] + HashAggregate [ss_sold_date_sk] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count,count,max,max,max,max,max,max,max,max,max,count,count,max,max,max,max,max,max,max,max,max] InputAdapter Exchange [ss_sold_date_sk] #2 WholeStageCodegen (1) - HashAggregate [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_sold_time_sk,ss_store_sk] [count,count,count,count,count(1),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk)] + HashAggregate [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count,count,max,max,max,max,max,max,max,max,max,count,count,max,max,max,max,max,max,max,max,max] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max/simplified.txt index 31622e85a4a03..9883e30a3e4b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/ss_max/simplified.txt @@ -1,14 +1,14 @@ WholeStageCodegen (3) - HashAggregate [count,count,count,max,max,max,max,max,max,max,max,max] [count,count,count,count(1),count(ss_sold_date_sk),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk),max_ss_addr_sk,max_ss_cdemo_sk,max_ss_customer_sk,max_ss_hdemo_sk,max_ss_item_sk,max_ss_promo_sk,max_ss_sold_date_sk,max_ss_sold_time_sk,max_ss_store_sk,not_null_total,total,unique_days] + HashAggregate [count,count,max,max,max,max,max,max,max,max,max,count] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count(ss_sold_date_sk),total,not_null_total,unique_days,max_ss_sold_date_sk,max_ss_sold_time_sk,max_ss_item_sk,max_ss_customer_sk,max_ss_cdemo_sk,max_ss_hdemo_sk,max_ss_addr_sk,max_ss_store_sk,max_ss_promo_sk,count,count,max,max,max,max,max,max,max,max,max,count] InputAdapter Exchange #1 WholeStageCodegen (2) - HashAggregate [ss_sold_date_sk] [count,count,count,count,count,count,count(1),count(ss_sold_date_sk),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk)] - HashAggregate [ss_sold_date_sk] [count,count,count,count,count(1),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk)] + HashAggregate [ss_sold_date_sk] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count(ss_sold_date_sk),count,count,max,max,max,max,max,max,max,max,max,count,count,count,max,max,max,max,max,max,max,max,max,count] + HashAggregate [ss_sold_date_sk] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count,count,max,max,max,max,max,max,max,max,max,count,count,max,max,max,max,max,max,max,max,max] InputAdapter Exchange [ss_sold_date_sk] #2 WholeStageCodegen (1) - HashAggregate [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_sold_time_sk,ss_store_sk] [count,count,count,count,count(1),count(ss_sold_date_sk),max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max,max(ss_addr_sk),max(ss_cdemo_sk),max(ss_customer_sk),max(ss_hdemo_sk),max(ss_item_sk),max(ss_promo_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_store_sk)] + HashAggregate [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk] [count(1),count(ss_sold_date_sk),max(ss_sold_date_sk),max(ss_sold_time_sk),max(ss_item_sk),max(ss_customer_sk),max(ss_cdemo_sk),max(ss_hdemo_sk),max(ss_addr_sk),max(ss_store_sk),max(ss_promo_sk),count,count,max,max,max,max,max,max,max,max,max,count,count,max,max,max,max,max,max,max,max,max] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt index 40992f74d8a06..99515b47285db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (12) Project [c_customer_id] - SortMergeJoin [c_customer_sk,ctr_customer_sk] + SortMergeJoin [ctr_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (9) Sort [ctr_customer_sk] @@ -11,24 +11,24 @@ TakeOrderedAndProject [c_customer_id] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_store_sk,ctr_store_skL,ctr_total_return] + BroadcastHashJoin [ctr_store_sk,ctr_store_skL,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sum,sum(UnscaledValue(sr_return_amt))] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] InputAdapter Exchange [sr_customer_sk,sr_store_sk] #2 WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_return_amt,sr_store_sk] [sum,sum] - Project [sr_customer_sk,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Filter [sr_customer_sk,sr_returned_date_sk,sr_store_sk] + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_returned_date_sk,sr_store_sk,sr_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_customer_sk,sr_store_sk,sr_return_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -36,22 +36,22 @@ TakeOrderedAndProject [c_customer_id] BroadcastExchange #4 WholeStageCodegen (6) Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [count,ctr_store_sk,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_store_skL,sum] + HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_store_skL,sum,count] InputAdapter Exchange [ctr_store_sk] #5 WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [count,count,sum,sum] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [ctr_store_sk,ctr_total_return,sum,sum(UnscaledValue(sr_return_amt))] + HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] InputAdapter Exchange [sr_customer_sk,sr_store_sk] #6 WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_return_amt,sr_store_sk] [sum,sum] - Project [sr_customer_sk,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_customer_sk,sr_store_sk,sr_return_amt] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -61,7 +61,7 @@ TakeOrderedAndProject [c_customer_id] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter WholeStageCodegen (11) Sort [c_customer_sk] @@ -71,4 +71,4 @@ TakeOrderedAndProject [c_customer_id] Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_customer_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 301ea242ec70b..6868eb76666e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [c_customer_id] WholeStageCodegen (9) Project [c_customer_id] - BroadcastHashJoin [c_customer_sk,ctr_customer_sk] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] BroadcastHashJoin [ctr_store_sk,s_store_sk] Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_store_sk,ctr_store_skL,ctr_total_return] + BroadcastHashJoin [ctr_store_sk,ctr_store_skL,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sum,sum(UnscaledValue(sr_return_amt))] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] InputAdapter Exchange [sr_customer_sk,sr_store_sk] #1 WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_return_amt,sr_store_sk] [sum,sum] - Project [sr_customer_sk,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Filter [sr_customer_sk,sr_returned_date_sk,sr_store_sk] + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_returned_date_sk,sr_store_sk,sr_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_customer_sk,sr_store_sk,sr_return_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -30,22 +30,22 @@ TakeOrderedAndProject [c_customer_id] BroadcastExchange #3 WholeStageCodegen (6) Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [count,ctr_store_sk,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_store_skL,sum] + HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_store_skL,sum,count] InputAdapter Exchange [ctr_store_sk] #4 WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [count,count,sum,sum] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [ctr_store_sk,ctr_total_return,sum,sum(UnscaledValue(sr_return_amt))] + HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] InputAdapter Exchange [sr_customer_sk,sr_store_sk] #5 WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_return_amt,sr_store_sk] [sum,sum] - Project [sr_customer_sk,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_customer_sk,sr_store_sk,sr_return_amt] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -55,11 +55,11 @@ TakeOrderedAndProject [c_customer_id] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_customer_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt index 32dfbc7173529..85b7f1195a257 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/simplified.txt @@ -1,11 +1,11 @@ -TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] +TakeOrderedAndProject [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,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (18) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count,count(1)] + HashAggregate [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] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter - Exchange [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [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] #1 WholeStageCodegen (17) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [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,count] + Project [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] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (14) @@ -15,7 +15,7 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep WholeStageCodegen (13) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] @@ -29,37 +29,37 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (4) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (8) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #6 WholeStageCodegen (7) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (11) @@ -72,14 +72,14 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #8 WholeStageCodegen (12) Project [ca_address_sk] - Filter [ca_address_sk,ca_county] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] @@ -92,4 +92,4 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 56c1a25a07c5c..509ce3ac4837f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -83,7 +83,7 @@ Condition : isnotnull(ss_sold_date_sk#6) Output [3]: [d_date_sk#8, d_year#9, d_moy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] @@ -91,7 +91,7 @@ Input [3]: [d_date_sk#8, d_year#9, d_moy#10] (9) Filter [codegen id : 1] Input [3]: [d_date_sk#8, d_year#9, d_moy#10] -Condition : (((((isnotnull(d_moy#10) AND isnotnull(d_year#9)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 1)) AND (d_moy#10 <= 4)) AND isnotnull(d_date_sk#8)) +Condition : (((((isnotnull(d_year#9) AND isnotnull(d_moy#10)) AND (d_year#9 = 2002)) AND (d_moy#10 >= 1)) AND (d_moy#10 <= 4)) AND isnotnull(d_date_sk#8)) (10) Project [codegen id : 1] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 19781433465a3..2eb1bd3c57306 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] +TakeOrderedAndProject [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,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (10) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count,count(1)] + HashAggregate [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] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter - Exchange [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [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] #1 WholeStageCodegen (9) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [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,count] + Project [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] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] @@ -17,33 +17,33 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -54,14 +54,14 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) Project [ca_address_sk] - Filter [ca_address_sk,ca_county] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] @@ -71,4 +71,4 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt index 281afc27939f3..72588fa138fc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt @@ -4,11 +4,11 @@ TakeOrderedAndProject [customer_preferred_cust_flag] SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) - Project [customer_id,customer_preferred_cust_flag,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (8) @@ -17,29 +17,29 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Exchange [customer_id] #1 WholeStageCodegen (7) Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #2 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #2 WholeStageCodegen (6) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #3 WholeStageCodegen (2) - Project [d_year,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -49,39 +49,39 @@ TakeOrderedAndProject [customer_preferred_cust_flag] InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (4) - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter WholeStageCodegen (16) Sort [customer_id] InputAdapter Exchange [customer_id] #6 WholeStageCodegen (15) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,customer_preferred_cust_flag,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,customer_preferred_cust_flag,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #7 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #7 WholeStageCodegen (14) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (10) - Project [d_year,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -89,7 +89,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (13) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (25) Sort [customer_id] @@ -98,61 +98,61 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (24) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #11 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (23) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (20) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #12 WholeStageCodegen (19) - Project [d_year,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #14 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (32) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (29) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 WholeStageCodegen (28) - Project [d_year,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #9 InputAdapter WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 6cb07efba16af..3850cea62922e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -2,60 +2,60 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (16) Project [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,customer_preferred_cust_flag,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #1 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 WholeStageCodegen (3) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_list_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,customer_preferred_cust_flag,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,customer_preferred_cust_flag,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #5 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 WholeStageCodegen (6) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_list_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -64,45 +64,45 @@ TakeOrderedAndProject [customer_preferred_cust_flag] WholeStageCodegen (11) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #8 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8 WholeStageCodegen (10) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_list_price] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #10 WholeStageCodegen (15) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #11 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (14) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_list_price] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #9 + ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt index cf472842f1431..e9b94a6b4651c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -8,25 +8,25 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ws_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (6) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] - SortMergeJoin [i_item_sk,ws_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (2) - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) @@ -34,7 +34,7 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -44,4 +44,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index 620baa8d07fa8..f67bfe65f114c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -8,26 +8,26 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ws_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (3) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -35,4 +35,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index 5a0627040acd4..e410b27e9cf3b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt @@ -1,30 +1,30 @@ WholeStageCodegen (7) - HashAggregate [count,count,count,sum,sum,sum,sum] [avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),avg(cast(ss_quantity as bigint)),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),avg(ss_quantity),count,count,count,sum,sum,sum,sum,sum(UnscaledValue(ss_ext_wholesale_cost)),sum(ss_ext_wholesale_cost)] + HashAggregate [sum,count,sum,count,sum,count,sum] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] InputAdapter Exchange #1 WholeStageCodegen (6) - HashAggregate [ss_ext_sales_price,ss_ext_wholesale_cost,ss_quantity] [count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_ext_wholesale_cost,ss_quantity] - BroadcastHashJoin [ca_address_sk,ca_state,ss_addr_sk,ss_net_profit] - Project [ss_addr_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_quantity] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_quantity,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_education_status,cd_marital_status,hd_demo_sk,hd_dep_count,ss_hdemo_sk,ss_sales_price] - Project [cd_education_status,cd_marital_status,ss_addr_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,cd_education_status,cd_marital_status,ss_cdemo_sk,ss_sales_price] + HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [cd_marital_status,cd_education_status,ss_sold_date_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] + BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk,cd_marital_status,cd_education_status,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] - Filter [ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -36,7 +36,7 @@ WholeStageCodegen (7) BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -51,7 +51,7 @@ WholeStageCodegen (7) BroadcastExchange #6 WholeStageCodegen (5) Project [ca_address_sk,ca_state] - Filter [ca_address_sk,ca_country,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 8a0a2ee6b74c7..6e3a322d86765 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -1,23 +1,23 @@ WholeStageCodegen (7) - HashAggregate [count,count,count,sum,sum,sum,sum] [avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),avg(cast(ss_quantity as bigint)),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),avg(ss_quantity),count,count,count,sum,sum,sum,sum,sum(UnscaledValue(ss_ext_wholesale_cost)),sum(ss_ext_wholesale_cost)] + HashAggregate [sum,count,sum,count,sum,count,sum] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] InputAdapter Exchange #1 WholeStageCodegen (6) - HashAggregate [ss_ext_sales_price,ss_ext_wholesale_cost,ss_quantity] [count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_ext_wholesale_cost,ss_quantity] - BroadcastHashJoin [cd_education_status,cd_marital_status,hd_demo_sk,hd_dep_count,ss_hdemo_sk,ss_sales_price] - Project [cd_education_status,cd_marital_status,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [cd_demo_sk,cd_education_status,cd_marital_status,ss_cdemo_sk,ss_sales_price] - Project [ss_cdemo_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_cdemo_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_address_sk,ca_state,ss_addr_sk,ss_net_profit] - Project [ss_addr_sk,ss_cdemo_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_addr_sk,ss_sold_date_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_ext_sales_price,ss_ext_wholesale_cost,ss_hdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -29,25 +29,25 @@ WholeStageCodegen (7) BroadcastExchange #3 WholeStageCodegen (2) Project [ca_address_sk,ca_state] - Filter [ca_address_sk,ca_country,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 49cdab6de1e4f..c3e9f9418cef5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -158,7 +158,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -166,7 +166,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_class_id#8) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -217,7 +217,7 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, d_date_sk#10] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category_id), IsNotNull(i_brand_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] @@ -225,7 +225,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (21) Filter [codegen id : 4] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_category_id#9)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (22) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 71d4c167f6564..c6dbfcaa3fe43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,67 +1,67 @@ -TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sales),sum(sales)] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] WholeStageCodegen (119) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,spark_grouping_id,sum,sum] [isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sales),sum(sales)] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,spark_grouping_id] #1 + Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 WholeStageCodegen (118) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales,spark_grouping_id] [isEmpty,isEmpty,sum,sum,sum,sum] - Expand [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union WholeStageCodegen (39) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #16 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #17 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [d_date_sk] #17 WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #17 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #2 + Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (38) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_item_sk,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] WholeStageCodegen (2) @@ -72,72 +72,72 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] WholeStageCodegen (18) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (17) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (16) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #6 + Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (15) - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (6) - Sort [brand_id,category_id,class_id] + Sort [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #7 + Exchange [brand_id,class_id,category_id] #7 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #8 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #9 WholeStageCodegen (4) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (10) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #10 + Exchange [i_brand_id,i_class_id,i_category_id] #10 WholeStageCodegen (9) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #8 InputAdapter @@ -146,32 +146,32 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (14) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #12 + Exchange [i_brand_id,i_class_id,i_category_id] #12 WholeStageCodegen (13) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter BroadcastExchange #13 WholeStageCodegen (19) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #14 SortMergeJoin [i_item_sk,ss_item_sk] @@ -183,23 +183,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (37) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #4 WholeStageCodegen (78) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #18 + Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (77) - HashAggregate [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] [count,count,isEmpty,isEmpty,sum,sum] - Project [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_list_price,cs_quantity] + Project [cs_item_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] @@ -211,7 +211,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] WholeStageCodegen (57) Sort [ss_item_sk] InputAdapter @@ -219,22 +219,22 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa InputAdapter ReusedExchange [d_date_sk] #13 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #14 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 WholeStageCodegen (117) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #20 + Exchange [i_brand_id,i_class_id,i_category_id] #20 WholeStageCodegen (116) - HashAggregate [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_item_sk,ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter - SortMergeJoin [ss_item_sk,ws_item_sk] + SortMergeJoin [ws_item_sk,ss_item_sk] WholeStageCodegen (80) Sort [ws_item_sk] InputAdapter @@ -243,7 +243,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] WholeStageCodegen (96) Sort [ss_item_sk] InputAdapter @@ -251,4 +251,4 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa InputAdapter ReusedExchange [d_date_sk] #13 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #14 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 5332be30cb775..c1b77321f16e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -134,7 +134,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -142,7 +142,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) +Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -162,7 +162,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category_id), IsNotNull(i_brand_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 1] @@ -170,7 +170,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (12) Filter [codegen id : 1] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) +Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (13) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 9facf93115f1f..604bd792f5ffd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -1,149 +1,149 @@ -TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sales),sum(sales)] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,spark_grouping_id,sum,sum] [isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sales),sum(sales)] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,spark_grouping_id] #1 + Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales,spark_grouping_id] [isEmpty,isEmpty,sum,sum,sum,sum] - Expand [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union WholeStageCodegen (26) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #13 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #14 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [d_date_sk] #14 WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #14 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #2 + Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (25) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] BroadcastHashJoin [ss_item_sk,ss_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (11) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (10) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #5 + Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (9) - HashAggregate [brand_id,category_id,class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_category_id,i_class_id] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter BroadcastExchange #9 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #10 WholeStageCodegen (8) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #9 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter @@ -153,62 +153,62 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,sum(number_sa Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter BroadcastExchange #12 WholeStageCodegen (24) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (52) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #15 + Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (51) - HashAggregate [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] [count,count,isEmpty,isEmpty,sum,sum] - Project [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_list_price,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id] + Project [cs_sold_date_sk,cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] BroadcastHashJoin [cs_item_sk,ss_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter ReusedExchange [d_date_sk] #12 WholeStageCodegen (78) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #16 + Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (77) - HashAggregate [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - BroadcastHashJoin [ss_item_sk,ws_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 InputAdapter ReusedExchange [d_date_sk] #12 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 6d39c12011180..f71ceaaf91f47 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -137,7 +137,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -145,7 +145,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_class_id#8) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -196,7 +196,7 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, d_date_sk#10] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] @@ -204,7 +204,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (21) Filter [codegen id : 4] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (22) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] @@ -437,7 +437,7 @@ Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_d Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (73) ColumnarToRow [codegen id : 20] @@ -445,7 +445,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (74) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (75) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 5141d40e7325b..37186560cb3b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,59 +1,59 @@ -TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,number_sales,number_sales,sales,sales] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (78) - BroadcastHashJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id] - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #15 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [d_date_sk] #16 WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #16 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #1 + Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (38) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_item_sk,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] WholeStageCodegen (2) @@ -64,72 +64,72 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] WholeStageCodegen (18) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #3 WholeStageCodegen (17) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (16) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #5 + Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (15) - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (6) - Sort [brand_id,category_id,class_id] + Sort [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #6 + Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (10) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #9 + Exchange [i_brand_id,i_class_id,i_category_id] #9 WholeStageCodegen (9) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter @@ -138,36 +138,36 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (14) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #11 + Exchange [i_brand_id,i_class_id,i_category_id] #11 WholeStageCodegen (13) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #10 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #12 WholeStageCodegen (19) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #1 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] @@ -179,10 +179,10 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat InputAdapter Exchange [i_item_sk] #14 WholeStageCodegen (20) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (37) Sort [ss_item_sk] InputAdapter @@ -190,24 +190,24 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat InputAdapter BroadcastExchange #17 WholeStageCodegen (77) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #18 + Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (76) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_item_sk,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] WholeStageCodegen (40) Sort [ss_item_sk] InputAdapter - ReusedExchange [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] #2 WholeStageCodegen (56) Sort [ss_item_sk] InputAdapter @@ -216,16 +216,16 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat BroadcastExchange #19 WholeStageCodegen (57) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #3 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index dfe1892d80d61..b68ce0e9f2264 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -119,7 +119,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -127,7 +127,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) +Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -147,7 +147,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 1] @@ -155,7 +155,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (12) Filter [codegen id : 1] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) AND isnotnull(i_class_id#7)) +Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (13) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] @@ -368,7 +368,7 @@ Join condition: None Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] @@ -376,7 +376,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (60) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) +Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (61) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#25] @@ -719,7 +719,7 @@ Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] @@ -727,7 +727,7 @@ Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] (129) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Condition : (((((isnotnull(d_moy#86) AND isnotnull(d_year#11)) AND isnotnull(d_dom#87)) AND (d_year#11 = 2000)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 2000)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) (130) Project [codegen id : 1] Output [1]: [d_week_seq#28] @@ -746,7 +746,7 @@ Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquer Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (132) ColumnarToRow [codegen id : 1] @@ -754,7 +754,7 @@ Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] (133) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Condition : (((((isnotnull(d_moy#86) AND isnotnull(d_year#11)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 11)) (134) Project [codegen id : 1] Output [1]: [d_week_seq#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index f1668ea399807..6e6950d4cb33a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -1,204 +1,204 @@ -TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,number_sales,number_sales,sales,sales] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id] - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #12 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #13 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [d_date_sk] #13 WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #13 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #1 + Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (25) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] BroadcastHashJoin [ss_item_sk,ss_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (11) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (10) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #4 + Exchange [brand_id,class_id,category_id] #4 WholeStageCodegen (9) - HashAggregate [brand_id,category_id,class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_category_id,i_class_id] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter BroadcastExchange #8 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #9 WholeStageCodegen (8) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #8 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (23) BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #2 InputAdapter BroadcastExchange #11 WholeStageCodegen (24) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #1 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #14 WholeStageCodegen (51) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #15 + Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (50) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] BroadcastHashJoin [ss_item_sk,ss_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] InputAdapter ReusedExchange [ss_item_sk] #2 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #10 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #16 WholeStageCodegen (49) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #3 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt index e03fd039c07d7..cd452ad14548c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/simplified.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (11) - HashAggregate [ca_zip,sum] [sum,sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price)] + HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] InputAdapter Exchange [ca_zip] #1 WholeStageCodegen (10) HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [ca_zip,cs_sales_price] - SortMergeJoin [c_customer_sk,ca_state,ca_zip,cs_bill_customer_sk,cs_sales_price] + Project [cs_sales_price,ca_zip] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk,ca_zip,ca_state,cs_sales_price] InputAdapter WholeStageCodegen (3) Sort [cs_bill_customer_sk] @@ -18,15 +18,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter WholeStageCodegen (9) Sort [c_customer_sk] @@ -41,10 +41,10 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] InputAdapter Exchange [c_current_addr_sk] #5 WholeStageCodegen (4) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (7) Sort [ca_address_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index e6e98be4b438f..9cac0b46f88d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] WholeStageCodegen (5) - HashAggregate [ca_zip,sum] [sum,sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price)] + HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] InputAdapter Exchange [ca_zip] #1 WholeStageCodegen (4) HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [ca_zip,cs_sales_price] + Project [cs_sales_price,ca_zip] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ca_zip,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_state,ca_zip,cs_sales_price] - Project [c_current_addr_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + Project [cs_sold_date_sk,cs_sales_price,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + Project [cs_sold_date_sk,cs_sales_price,c_current_addr_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -33,7 +33,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt index dc78021b94fbf..ea9a0b27ff700 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/simplified.txt @@ -1,22 +1,22 @@ -TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] +TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] WholeStageCodegen (12) - HashAggregate [count,sum,sum] [count,count(cs_order_number),order count ,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),total net profit ,total shipping cost ] + HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 WholeStageCodegen (11) - HashAggregate [cs_order_number] [count,count,count(cs_order_number),sum,sum,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - HashAggregate [cs_order_number] [sum,sum,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - HashAggregate [cs_ext_ship_cost,cs_net_profit,cs_order_number] [sum,sum,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - Project [cs_ext_ship_cost,cs_net_profit,cs_order_number] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_date_sk] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_date_sk] - BroadcastHashJoin [ca_address_sk,cs_ship_addr_sk] + Project [cs_ship_date_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] InputAdapter - SortMergeJoin [cr_order_number,cs_order_number] + SortMergeJoin [cs_order_number,cr_order_number] WholeStageCodegen (5) - Project [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk] + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] InputAdapter SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] WholeStageCodegen (2) @@ -24,19 +24,19 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] InputAdapter Exchange [cs_order_number] #2 WholeStageCodegen (1) - Filter [cs_call_center_sk,cs_ship_addr_sk,cs_ship_date_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk,cs_warehouse_sk] + Scan parquet 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] WholeStageCodegen (4) Sort [cs_order_number] InputAdapter Exchange [cs_order_number] #3 WholeStageCodegen (3) - Project [cs_order_number,cs_warehouse_sk] + Project [cs_warehouse_sk,cs_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_order_number,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number] WholeStageCodegen (7) Sort [cr_order_number] InputAdapter @@ -49,7 +49,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #5 WholeStageCodegen (8) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -57,7 +57,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #6 WholeStageCodegen (9) Project [cc_call_center_sk] - Filter [cc_call_center_sk,cc_county] + Filter [cc_county,cc_call_center_sk] ColumnarToRow InputAdapter Scan parquet default.call_center [cc_call_center_sk,cc_county] @@ -68,4 +68,4 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt index cdf86a4813208..a044b05365f8e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] +TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] WholeStageCodegen (8) - HashAggregate [count,sum,sum] [count,count(cs_order_number),order count ,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),total net profit ,total shipping cost ] + HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 WholeStageCodegen (7) - HashAggregate [cs_order_number] [count,count,count(cs_order_number),sum,sum,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - HashAggregate [cs_order_number] [sum,sum,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] InputAdapter Exchange [cs_order_number] #2 WholeStageCodegen (6) - HashAggregate [cs_ext_ship_cost,cs_net_profit,cs_order_number] [sum,sum,sum,sum,sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - Project [cs_ext_ship_cost,cs_net_profit,cs_order_number] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number] - BroadcastHashJoin [ca_address_sk,cs_ship_addr_sk] - Project [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - BroadcastHashJoin [cr_order_number,cs_order_number] - Project [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk] + BroadcastHashJoin [cs_order_number,cr_order_number] + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] BroadcastHashJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] - Filter [cs_call_center_sk,cs_ship_addr_sk,cs_ship_date_sk] + Filter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_ship_addr_sk,cs_ship_date_sk,cs_warehouse_sk] + Scan parquet 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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [cs_order_number,cs_warehouse_sk] + Project [cs_warehouse_sk,cs_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_order_number,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -43,12 +43,12 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -56,7 +56,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #7 WholeStageCodegen (5) Project [cc_call_center_sk] - Filter [cc_call_center_sk,cc_county] + Filter [cc_county,cc_call_center_sk] ColumnarToRow InputAdapter Scan parquet default.call_center [cc_call_center_sk,cc_county] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index df221f16db96f..e24b656e843aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -171,11 +171,11 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s (26) Exchange Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint), 5), true, [id=#18] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#18] (27) Sort [codegen id : 8] Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] -Arguments: [cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST], false, 0 +Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.date_dim Output [2]: [d_date_sk#19, d_quarter_name#20] @@ -203,7 +203,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Output [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (34) ColumnarToRow @@ -211,7 +211,7 @@ Input [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_ (35) Filter Input [5]: [sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Condition : (((isnotnull(sr_ticket_number#25) AND isnotnull(sr_customer_sk#24)) AND isnotnull(sr_item_sk#23)) AND isnotnull(sr_returned_date_sk#22)) +Condition : (((isnotnull(sr_customer_sk#24) AND isnotnull(sr_item_sk#23)) AND isnotnull(sr_ticket_number#25)) AND isnotnull(sr_returned_date_sk#22)) (36) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cast(d_date_sk#19 as bigint)] @@ -224,15 +224,15 @@ Input [6]: [d_date_sk#19, sr_returned_date_sk#22, sr_item_sk#23, sr_customer_sk# (38) Exchange Input [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Arguments: hashpartitioning(sr_ticket_number#25, sr_item_sk#23, sr_customer_sk#24, 5), true, [id=#27] +Arguments: hashpartitioning(sr_customer_sk#24, sr_item_sk#23, sr_ticket_number#25, 5), true, [id=#27] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#23, sr_customer_sk#24, sr_ticket_number#25, sr_return_quantity#26] -Arguments: [sr_ticket_number#25 ASC NULLS FIRST, sr_item_sk#23 ASC NULLS FIRST, sr_customer_sk#24 ASC NULLS FIRST], false, 0 +Arguments: [sr_customer_sk#24 ASC NULLS FIRST, sr_item_sk#23 ASC NULLS FIRST, sr_ticket_number#25 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint)] -Right keys [3]: [sr_ticket_number#25, sr_item_sk#23, sr_customer_sk#24] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#24, sr_item_sk#23, sr_ticket_number#25] Join condition: None (41) Project [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index 1a53b72ed00c3..216adf3588eca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycount,as_store_returns_quantitystdev,catalog_sales_quantityave,catalog_sales_quantitycount,catalog_sales_quantitycov,catalog_sales_quantitystdev,i_item_desc,i_item_id,s_state,store_returns_quantitycov,store_sales_quantityave,store_sales_quantitycount,store_sales_quantitycov,store_sales_quantitystdev] +TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (18) - HashAggregate [avg,avg,avg,count,count,count,count,count,count,i_item_desc,i_item_id,m2,m2,m2,n,n,n,s_state,sum,sum,sum] [as_store_returns_quantityave,as_store_returns_quantitycount,as_store_returns_quantitystdev,avg,avg,avg,avg(cast(cs_quantity as bigint)),avg(cast(sr_return_quantity as bigint)),avg(cast(ss_quantity as bigint)),catalog_sales_quantityave,catalog_sales_quantitycount,catalog_sales_quantitycov,catalog_sales_quantitystdev,count,count,count,count,count,count,count(cs_quantity),count(sr_return_quantity),count(ss_quantity),m2,m2,m2,n,n,n,stddev_samp(cast(cs_quantity as double)),stddev_samp(cast(sr_return_quantity as double)),stddev_samp(cast(ss_quantity as double)),store_returns_quantitycov,store_sales_quantityave,store_sales_quantitycount,store_sales_quantitycov,store_sales_quantitystdev,sum,sum,sum] + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter - Exchange [i_item_desc,i_item_id,s_state] #1 + Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (17) - HashAggregate [cs_quantity,i_item_desc,i_item_id,s_state,sr_return_quantity,ss_quantity] [avg,avg,avg,avg,avg,avg,count,count,count,count,count,count,count,count,count,count,count,count,m2,m2,m2,m2,m2,m2,n,n,n,n,n,n,sum,sum,sum,sum,sum,sum] - Project [cs_quantity,i_item_desc,i_item_id,s_state,sr_return_quantity,ss_quantity] - SortMergeJoin [cs_bill_customer_sk,cs_item_sk,sr_customer_sk,sr_item_sk] + HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (13) Sort [sr_customer_sk,sr_item_sk] InputAdapter Exchange [sr_customer_sk,sr_item_sk] #2 WholeStageCodegen (12) - Project [i_item_desc,i_item_id,s_state,sr_customer_sk,sr_item_sk,sr_return_quantity,ss_quantity] - SortMergeJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] + Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (8) Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 WholeStageCodegen (7) - Project [i_item_desc,i_item_id,s_state,ss_customer_sk,ss_item_sk,ss_quantity,ss_ticket_number] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (3) - Project [s_state,ss_customer_sk,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_quantity,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_quarter_name] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] @@ -51,7 +51,7 @@ TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycou Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] @@ -61,27 +61,27 @@ TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycou Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] InputAdapter WholeStageCodegen (11) Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) - Project [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_ticket_number] + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [d_date_sk,sr_returned_date_sk] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) Project [d_date_sk] - Filter [d_date_sk,d_quarter_name] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] @@ -93,6 +93,6 @@ TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycou Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index e04b67a75d62f..4085b4ab988cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -53,7 +53,7 @@ TakeOrderedAndProject (48) Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 8] @@ -61,13 +61,13 @@ Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss (3) Filter [codegen id : 8] Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] -Condition : ((((isnotnull(ss_item_sk#2) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) +Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) (4) Scan parquet default.store_returns Output [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -75,7 +75,7 @@ Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_num (6) Filter [codegen id : 1] Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Condition : (((isnotnull(sr_ticket_number#10) AND isnotnull(sr_customer_sk#9)) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_returned_date_sk#7)) +Condition : (((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_returned_date_sk#7)) (7) BroadcastExchange Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 1e346110a5348..e9b95747c294f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycount,as_store_returns_quantitystdev,catalog_sales_quantityave,catalog_sales_quantitycount,catalog_sales_quantitycov,catalog_sales_quantitystdev,i_item_desc,i_item_id,s_state,store_returns_quantitycov,store_sales_quantityave,store_sales_quantitycount,store_sales_quantitycov,store_sales_quantitystdev] +TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (9) - HashAggregate [avg,avg,avg,count,count,count,count,count,count,i_item_desc,i_item_id,m2,m2,m2,n,n,n,s_state,sum,sum,sum] [as_store_returns_quantityave,as_store_returns_quantitycount,as_store_returns_quantitystdev,avg,avg,avg,avg(cast(cs_quantity as bigint)),avg(cast(sr_return_quantity as bigint)),avg(cast(ss_quantity as bigint)),catalog_sales_quantityave,catalog_sales_quantitycount,catalog_sales_quantitycov,catalog_sales_quantitystdev,count,count,count,count,count,count,count(cs_quantity),count(sr_return_quantity),count(ss_quantity),m2,m2,m2,n,n,n,stddev_samp(cast(cs_quantity as double)),stddev_samp(cast(sr_return_quantity as double)),stddev_samp(cast(ss_quantity as double)),store_returns_quantitycov,store_sales_quantityave,store_sales_quantitycount,store_sales_quantitycov,store_sales_quantitystdev,sum,sum,sum] + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter - Exchange [i_item_desc,i_item_id,s_state] #1 + Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (8) - HashAggregate [cs_quantity,i_item_desc,i_item_id,s_state,sr_return_quantity,ss_quantity] [avg,avg,avg,avg,avg,avg,count,count,count,count,count,count,count,count,count,count,count,count,m2,m2,m2,m2,m2,m2,n,n,n,n,n,n,sum,sum,sum,sum,sum,sum] - Project [cs_quantity,i_item_desc,i_item_id,s_state,sr_return_quantity,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [cs_quantity,s_state,sr_return_quantity,ss_item_sk,ss_quantity] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [cs_quantity,sr_return_quantity,ss_item_sk,ss_quantity,ss_store_sk] + HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_sold_date_sk,sr_return_quantity,ss_item_sk,ss_quantity,ss_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [cs_quantity,cs_sold_date_sk,sr_return_quantity,sr_returned_date_sk,ss_item_sk,ss_quantity,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [cs_quantity,cs_sold_date_sk,sr_return_quantity,sr_returned_date_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,sr_customer_sk,sr_item_sk] - Project [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_returned_date_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_sold_date_sk,cs_quantity] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_quarter_name] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] @@ -49,7 +49,7 @@ TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycou BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_quarter_name] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] @@ -61,11 +61,11 @@ TakeOrderedAndProject [as_store_returns_quantityave,as_store_returns_quantitycou Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt index cdc4bf9d37284..8c76e7cab3310 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt @@ -1,42 +1,42 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,ca_state,i_item_id] +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] WholeStageCodegen (14) - HashAggregate [ca_country,ca_county,ca_state,count,count,count,count,count,count,count,i_item_id,spark_grouping_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [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] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [ca_country,ca_county,ca_state,i_item_id,spark_grouping_id] #1 + Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 WholeStageCodegen (13) - HashAggregate [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id,spark_grouping_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Expand [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [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] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [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] + Project [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] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (5) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 WholeStageCodegen (4) - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [cd_demo_sk,cd_dep_count] - Filter [cd_demo_sk,cd_education_status,cd_gender] + Filter [cd_gender,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_count,cd_education_status,cd_gender] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -46,14 +46,14 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (11) - Project [c_birth_year,c_customer_sk,ca_country,ca_county,ca_state] + Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (8) @@ -61,20 +61,20 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Exchange [c_current_cdemo_sk] #7 WholeStageCodegen (7) - Project [c_birth_year,c_current_cdemo_sk,c_customer_sk,ca_country,ca_county,ca_state] + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_county,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter WholeStageCodegen (10) Sort [cd_demo_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 7e967bf60236e..0d4c8cb7f985f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -1,43 +1,43 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,ca_state,i_item_id] +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] WholeStageCodegen (8) - HashAggregate [ca_country,ca_county,ca_state,count,count,count,count,count,count,count,i_item_id,spark_grouping_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [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] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [ca_country,ca_county,ca_state,i_item_id,spark_grouping_id] #1 + Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 WholeStageCodegen (7) - HashAggregate [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id,spark_grouping_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Expand [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [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] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [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] + Project [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] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk,cd_dep_count] - Filter [cd_demo_sk,cd_education_status,cd_gender] + Filter [cd_gender,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_count,cd_education_status,cd_gender] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -48,15 +48,15 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_county,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -66,4 +66,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt index 1a285ae598c44..c8737d8a70782 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] +TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (13) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter - Exchange [i_brand,i_brand_id,i_manufact,i_manufact_id] #1 + Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 WholeStageCodegen (12) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] - SortMergeJoin [c_customer_sk,ca_zip,s_zip,ss_customer_sk] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] InputAdapter WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #2 WholeStageCodegen (4) - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,s_zip,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_customer_sk,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ss_store_sk] + Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_customer_sk,ss_ext_sales_price,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand_id,i_brand,i_manufact_id,i_manufact,ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk,i_manufact,i_manufact_id] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id,i_manufact,i_manufact_id] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Filter [s_store_sk,s_zip] + Filter [s_zip,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_zip] @@ -52,8 +52,8 @@ TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (10) - Project [c_customer_sk,ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] + Project [ca_zip,c_customer_sk] + SortMergeJoin [ca_address_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (7) Sort [ca_address_sk] @@ -70,7 +70,7 @@ TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] InputAdapter Exchange [c_current_addr_sk] #8 WholeStageCodegen (8) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index 9217520556863..1bbbf35e4d510 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] +TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (7) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter - Exchange [i_brand,i_brand_id,i_manufact,i_manufact_id] #1 + Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 WholeStageCodegen (6) - HashAggregate [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price] - BroadcastHashJoin [ca_zip,s_store_sk,s_zip,ss_store_sk] - Project [ca_zip,i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price,ss_store_sk] + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,i_brand,i_brand_id,i_manufact,i_manufact_id,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_brand,i_brand_id,i_manufact,i_manufact_id,ss_customer_sk,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_store_sk] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk,i_manufact,i_manufact_id] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id,i_manufact,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -52,7 +52,7 @@ TakeOrderedAndProject [brand,brand_id,ext_price,i_manufact,i_manufact_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [s_store_sk,s_zip] + Filter [s_zip,s_store_sk] ColumnarToRow InputAdapter Scan parquet default.store [s_store_sk,s_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index f7d6dce3fbf97..3df7e4c8e6f3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -3,7 +3,7 @@ WholeStageCodegen (16) InputAdapter Exchange [d_week_seq1] #1 WholeStageCodegen (15) - Project [d_week_seq1,fri_sales1,fri_sales2,mon_sales1,mon_sales2,sat_sales1,sat_sales2,sun_sales1,sun_sales2,thu_sales1,thu_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2] + Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] SortMergeJoin [d_week_seq1,d_week_seq2] InputAdapter WholeStageCodegen (7) @@ -11,41 +11,41 @@ WholeStageCodegen (16) InputAdapter Exchange [d_week_seq1] #2 WholeStageCodegen (6) - Project [d_week_seq,fri_sales,mon_sales,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #3 WholeStageCodegen (4) - HashAggregate [d_day_name,d_week_seq,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,sales_price] - BroadcastHashJoin [d_date_sk,sold_date_sk] + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (1) - Project [ws_ext_sales_price,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] WholeStageCodegen (2) - Project [cs_ext_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_ext_sales_price] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Filter [d_date_sk,d_week_seq] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) Project [d_week_seq] - Filter [d_week_seq,d_year] + Filter [d_year,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_week_seq,d_year] @@ -55,16 +55,16 @@ WholeStageCodegen (16) InputAdapter Exchange [d_week_seq2] #6 WholeStageCodegen (13) - Project [d_week_seq,fri_sales,mon_sales,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #3 InputAdapter BroadcastExchange #7 WholeStageCodegen (12) Project [d_week_seq] - Filter [d_week_seq,d_year] + Filter [d_year,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_week_seq,d_year] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 2e2d1e53fda09..424a535e14847 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -3,59 +3,59 @@ WholeStageCodegen (13) InputAdapter Exchange [d_week_seq1] #1 WholeStageCodegen (12) - Project [d_week_seq1,fri_sales1,fri_sales2,mon_sales1,mon_sales2,sat_sales1,sat_sales2,sun_sales1,sun_sales2,thu_sales1,thu_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2] + Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [d_week_seq1,d_week_seq2] - Project [d_week_seq,fri_sales,mon_sales,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #2 WholeStageCodegen (4) - HashAggregate [d_day_name,d_week_seq,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,sales_price] - BroadcastHashJoin [d_date_sk,sold_date_sk] + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (1) - Project [ws_ext_sales_price,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] WholeStageCodegen (2) - Project [cs_ext_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_ext_sales_price] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) Filter [d_date_sk,d_week_seq] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (5) Project [d_week_seq] - Filter [d_week_seq,d_year] + Filter [d_year,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 WholeStageCodegen (11) - Project [d_week_seq,fri_sales,mon_sales,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (10) Project [d_week_seq] - Filter [d_week_seq,d_year] + Filter [d_year,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_week_seq,d_year] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt index b7d7a77003325..1e8ab18f5e21a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -8,12 +8,12 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(cs_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (6) - HashAggregate [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] [sum,sum] - Project [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) @@ -21,12 +21,12 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (2) - Project [cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) @@ -34,7 +34,7 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -44,4 +44,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index b462752d01701..6259c1d53f19c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -8,26 +8,26 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(cs_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (3) - HashAggregate [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] [sum,sum] - Project [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_category,i_class,i_current_price,i_item_desc,i_item_id] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -35,4 +35,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt index f862d4135937d..0ee47d05af65b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/simplified.txt @@ -1,40 +1,40 @@ -TakeOrderedAndProject [i_item_id,inv_after,inv_before,w_warehouse_name] +TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (5) - Filter [inv_after,inv_before] - HashAggregate [i_item_id,sum,sum,w_warehouse_name] [inv_after,inv_before,sum,sum,sum(cast(CASE WHEN (d_date < 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),sum(cast(CASE WHEN (d_date >= 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint))] + Filter [inv_before,inv_after] + HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(cast(CASE WHEN (d_date < 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),sum(cast(CASE WHEN (d_date >= 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),inv_before,inv_after,sum,sum] InputAdapter - Exchange [i_item_id,w_warehouse_name] #1 + Exchange [w_warehouse_name,i_item_id] #1 WholeStageCodegen (4) - HashAggregate [d_date,i_item_id,inv_quantity_on_hand,w_warehouse_name] [sum,sum,sum,sum] - Project [d_date,i_item_id,inv_quantity_on_hand,w_warehouse_name] + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_date,i_item_id,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_item_id,inv_date_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_warehouse_sk,inv_item_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_id,i_item_sk] + Project [i_item_sk,i_item_id] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_current_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index d814563539bad..9b5483bd7191b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -1,40 +1,40 @@ -TakeOrderedAndProject [i_item_id,inv_after,inv_before,w_warehouse_name] +TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] WholeStageCodegen (5) - Filter [inv_after,inv_before] - HashAggregate [i_item_id,sum,sum,w_warehouse_name] [inv_after,inv_before,sum,sum,sum(cast(CASE WHEN (d_date < 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),sum(cast(CASE WHEN (d_date >= 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint))] + Filter [inv_before,inv_after] + HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(cast(CASE WHEN (d_date < 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),sum(cast(CASE WHEN (d_date >= 11027) THEN inv_quantity_on_hand ELSE 0 END as bigint)),inv_before,inv_after,sum,sum] InputAdapter - Exchange [i_item_id,w_warehouse_name] #1 + Exchange [w_warehouse_name,i_item_id] #1 WholeStageCodegen (4) - HashAggregate [d_date,i_item_id,inv_quantity_on_hand,w_warehouse_name] [sum,sum,sum,sum] - Project [d_date,i_item_id,inv_quantity_on_hand,w_warehouse_name] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_item_id,inv_date_sk,inv_quantity_on_hand,w_warehouse_name] - BroadcastHashJoin [i_item_sk,inv_item_sk] + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_quantity_on_hand,w_warehouse_name,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Filter [inv_warehouse_sk,inv_item_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_item_id,i_item_sk] + Project [i_item_sk,i_item_id] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_current_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/simplified.txt index 784510aca5a2f..2c4258c9d48e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/simplified.txt @@ -1,13 +1,13 @@ -TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (8) - HashAggregate [count,i_brand,i_category,i_class,i_product_name,spark_grouping_id,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name,spark_grouping_id] #1 + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 WholeStageCodegen (7) - HashAggregate [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand,spark_grouping_id] [count,count,sum,sum] - Expand [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] - SortMergeJoin [i_item_sk,inv_item_sk] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + SortMergeJoin [inv_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [inv_item_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Exchange [inv_item_sk] #2 WholeStageCodegen (3) Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [d_date_sk,inv_date_sk] + BroadcastHashJoin [inv_date_sk,d_date_sk] Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -33,7 +33,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -46,4 +46,4 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index cf0e275812deb..233babdf3f2e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -1,26 +1,26 @@ -TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (5) - HashAggregate [count,i_brand,i_category,i_class,i_product_name,spark_grouping_id,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name,spark_grouping_id] #1 + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 WholeStageCodegen (4) - HashAggregate [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand,spark_grouping_id] [count,count,sum,sum] - Expand [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt index 7b08c6a571b4c..9ee444cdd988c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt @@ -1,24 +1,24 @@ CollectLimit WholeStageCodegen (38) - HashAggregate [isEmpty,sum] [isEmpty,sum,sum(sales),sum(sales)] + HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 WholeStageCodegen (37) - HashAggregate [sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (18) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_list_price,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] WholeStageCodegen (11) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 WholeStageCodegen (10) - Project [cs_bill_customer_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] InputAdapter SortMergeJoin [cs_item_sk,item_sk] WholeStageCodegen (2) @@ -29,7 +29,7 @@ CollectLimit Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] WholeStageCodegen (9) Sort [item_sk] InputAdapter @@ -37,30 +37,30 @@ CollectLimit WholeStageCodegen (8) Project [item_sk] Filter [count(1)] - HashAggregate [count,d_date,i_item_sk,substr(i_item_desc, 1, 30)] [count,count(1),count(1),item_sk] - HashAggregate [d_date,i_item_desc,i_item_sk] [count,count,substr(i_item_desc, 1, 30)] - Project [d_date,i_item_desc,i_item_sk] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (5) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 WholeStageCodegen (4) - Project [d_date,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_year] + Project [d_date_sk,d_date] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_year] InputAdapter WholeStageCodegen (7) Sort [i_item_sk] @@ -70,22 +70,22 @@ CollectLimit Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] WholeStageCodegen (16) Sort [c_customer_sk] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (7) - HashAggregate [max] [max,max(csales),tpcds_cmax] + HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter Exchange #10 WholeStageCodegen (6) HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,isEmpty,sum] [csales,isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] @@ -93,16 +93,16 @@ CollectLimit Exchange [ss_customer_sk] #11 WholeStageCodegen (2) Project [ss_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #12 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -116,10 +116,10 @@ CollectLimit ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk] - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (13) Sort [ss_customer_sk] @@ -144,24 +144,24 @@ CollectLimit BroadcastExchange #14 WholeStageCodegen (17) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (36) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_list_price,ws_quantity,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] WholeStageCodegen (29) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 WholeStageCodegen (28) - Project [ws_bill_customer_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] InputAdapter - SortMergeJoin [item_sk,ws_item_sk] + SortMergeJoin [ws_item_sk,item_sk] WholeStageCodegen (20) Sort [ws_item_sk] InputAdapter @@ -170,7 +170,7 @@ CollectLimit Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] WholeStageCodegen (27) Sort [item_sk] InputAdapter @@ -180,10 +180,10 @@ CollectLimit Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (31) Sort [ss_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index a69293edd6218..d860e18574f2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -1,56 +1,56 @@ CollectLimit WholeStageCodegen (20) - HashAggregate [isEmpty,sum] [isEmpty,sum,sum(sales),sum(sales)] + HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 WholeStageCodegen (19) - HashAggregate [sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (9) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_list_price,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cs_bill_customer_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_quantity,cs_list_price] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_item_sk,item_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) Project [item_sk] Filter [count(1)] - HashAggregate [count,d_date,i_item_sk,substr(i_item_desc, 1, 30)] [count,count(1),count(1),item_sk] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] InputAdapter - Exchange [d_date,i_item_sk,substr(i_item_desc, 1, 30)] #3 + Exchange [substr(i_item_desc, 1, 30),i_item_sk,d_date] #3 WholeStageCodegen (3) - HashAggregate [d_date,i_item_desc,i_item_sk] [count,count,substr(i_item_desc, 1, 30)] - Project [d_date,i_item_desc,i_item_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_date,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_year] + Project [d_date_sk,d_date] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_year] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter BroadcastExchange #6 WholeStageCodegen (7) @@ -58,24 +58,24 @@ CollectLimit Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (5) - HashAggregate [max] [max,max(csales),tpcds_cmax] + HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter Exchange #9 WholeStageCodegen (4) HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,isEmpty,sum] [csales,isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] InputAdapter Exchange [c_customer_sk] #10 WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) @@ -87,17 +87,17 @@ CollectLimit BroadcastExchange #12 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter Exchange [c_customer_sk] #7 WholeStageCodegen (6) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [ss_customer_sk] ColumnarToRow InputAdapter @@ -113,21 +113,21 @@ CollectLimit BroadcastExchange #13 WholeStageCodegen (8) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (18) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_list_price,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Project [ws_bill_customer_sk,ws_list_price,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [item_sk,ws_item_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_item_sk,item_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [item_sk] #2 InputAdapter @@ -136,8 +136,8 @@ CollectLimit Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,isEmpty,sum] #7 + ReusedExchange [c_customer_sk,sum,isEmpty] #7 InputAdapter ReusedExchange [d_date_sk] #13 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt index 3964f50895888..e8891f032a091 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt @@ -1,25 +1,25 @@ -TakeOrderedAndProject [c_first_name,c_last_name,sales] +TakeOrderedAndProject [c_last_name,c_first_name,sales] Union WholeStageCodegen (28) - HashAggregate [c_first_name,c_last_name,isEmpty,sum] [isEmpty,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name] #1 + Exchange [c_last_name,c_first_name] #1 WholeStageCodegen (27) - HashAggregate [c_first_name,c_last_name,cs_list_price,cs_quantity] [isEmpty,isEmpty,sum,sum] - Project [c_first_name,c_last_name,cs_list_price,cs_quantity] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_list_price,c_first_name,c_last_name] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (18) - Project [cs_bill_customer_sk,cs_list_price,cs_quantity] + Project [cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] WholeStageCodegen (11) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 WholeStageCodegen (10) - Project [cs_bill_customer_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] InputAdapter SortMergeJoin [cs_item_sk,item_sk] WholeStageCodegen (2) @@ -30,7 +30,7 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] WholeStageCodegen (9) Sort [item_sk] InputAdapter @@ -38,30 +38,30 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] WholeStageCodegen (8) Project [item_sk] Filter [count(1)] - HashAggregate [count,d_date,i_item_sk,substr(i_item_desc, 1, 30)] [count,count(1),count(1),item_sk] - HashAggregate [d_date,i_item_desc,i_item_sk] [count,count,substr(i_item_desc, 1, 30)] - Project [d_date,i_item_desc,i_item_sk] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (5) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 WholeStageCodegen (4) - Project [d_date,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_year] + Project [d_date_sk,d_date] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_year] InputAdapter WholeStageCodegen (7) Sort [i_item_sk] @@ -71,22 +71,22 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] WholeStageCodegen (16) Sort [c_customer_sk] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (7) - HashAggregate [max] [max,max(csales),tpcds_cmax] + HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter Exchange #10 WholeStageCodegen (6) HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,isEmpty,sum] [csales,isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] @@ -94,16 +94,16 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Exchange [ss_customer_sk] #11 WholeStageCodegen (2) Project [ss_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #12 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -117,10 +117,10 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk] - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (13) Sort [ss_customer_sk] @@ -145,10 +145,10 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] BroadcastExchange #14 WholeStageCodegen (17) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter SortMergeJoin [c_customer_sk,c_customer_sk] WholeStageCodegen (20) @@ -168,10 +168,10 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (22) Sort [ss_customer_sk] @@ -183,27 +183,27 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] InputAdapter ReusedExchange [c_customer_sk] #9 WholeStageCodegen (56) - HashAggregate [c_first_name,c_last_name,isEmpty,sum] [isEmpty,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name] #17 + Exchange [c_last_name,c_first_name] #17 WholeStageCodegen (55) - HashAggregate [c_first_name,c_last_name,ws_list_price,ws_quantity] [isEmpty,isEmpty,sum,sum] - Project [c_first_name,c_last_name,ws_list_price,ws_quantity] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_list_price,c_first_name,c_last_name] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (46) - Project [ws_bill_customer_sk,ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] WholeStageCodegen (39) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #18 WholeStageCodegen (38) - Project [ws_bill_customer_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] InputAdapter - SortMergeJoin [item_sk,ws_item_sk] + SortMergeJoin [ws_item_sk,item_sk] WholeStageCodegen (30) Sort [ws_item_sk] InputAdapter @@ -212,7 +212,7 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] WholeStageCodegen (37) Sort [item_sk] InputAdapter @@ -222,10 +222,10 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (41) Sort [ss_customer_sk] @@ -252,10 +252,10 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (50) Sort [ss_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 277966b363866..f879f38d556e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,55 +1,55 @@ -TakeOrderedAndProject [c_first_name,c_last_name,sales] +TakeOrderedAndProject [c_last_name,c_first_name,sales] Union WholeStageCodegen (14) - HashAggregate [c_first_name,c_last_name,isEmpty,sum] [isEmpty,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name] #1 + Exchange [c_last_name,c_first_name] #1 WholeStageCodegen (13) - HashAggregate [c_first_name,c_last_name,cs_list_price,cs_quantity] [isEmpty,isEmpty,sum,sum] - Project [c_first_name,c_last_name,cs_list_price,cs_quantity] + HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_first_name,c_last_name,cs_list_price,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cs_bill_customer_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_quantity,cs_list_price,c_first_name,c_last_name] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_item_sk,item_sk] Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) Project [item_sk] Filter [count(1)] - HashAggregate [count,d_date,i_item_sk,substr(i_item_desc, 1, 30)] [count,count(1),count(1),item_sk] + HashAggregate [substr(i_item_desc, 1, 30),i_item_sk,d_date,count] [count(1),item_sk,count(1),count] InputAdapter - Exchange [d_date,i_item_sk,substr(i_item_desc, 1, 30)] #3 + Exchange [substr(i_item_desc, 1, 30),i_item_sk,d_date] #3 WholeStageCodegen (3) - HashAggregate [d_date,i_item_desc,i_item_sk] [count,count,substr(i_item_desc, 1, 30)] - Project [d_date,i_item_desc,i_item_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_date,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk] + HashAggregate [i_item_desc,i_item_sk,d_date] [count,substr(i_item_desc, 1, 30),count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_year] + Project [d_date_sk,d_date] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_year] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter BroadcastExchange #6 WholeStageCodegen (7) @@ -57,24 +57,24 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (5) - HashAggregate [max] [max,max(csales),tpcds_cmax] + HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter Exchange #9 WholeStageCodegen (4) HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,isEmpty,sum] [csales,isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),csales,sum,isEmpty] InputAdapter Exchange [c_customer_sk] #10 WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) @@ -86,17 +86,17 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] BroadcastExchange #12 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter Exchange [c_customer_sk] #7 WholeStageCodegen (6) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [c_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [ss_customer_sk] ColumnarToRow InputAdapter @@ -122,34 +122,34 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,isEmpty,sum] #7 + ReusedExchange [c_customer_sk,sum,isEmpty] #7 InputAdapter BroadcastExchange #15 WholeStageCodegen (12) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (28) - HashAggregate [c_first_name,c_last_name,isEmpty,sum] [isEmpty,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sales,sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name] #16 + Exchange [c_last_name,c_first_name] #16 WholeStageCodegen (27) - HashAggregate [c_first_name,c_last_name,ws_list_price,ws_quantity] [isEmpty,isEmpty,sum,sum] - Project [c_first_name,c_last_name,ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_first_name,c_last_name,ws_list_price,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Project [ws_bill_customer_sk,ws_list_price,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [item_sk,ws_item_sk] + HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_list_price,c_first_name,c_last_name] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_quantity,ws_list_price,c_first_name,c_last_name] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_item_sk,item_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [item_sk] #2 InputAdapter @@ -158,9 +158,9 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,isEmpty,sum] #7 + ReusedExchange [c_customer_sk,sum,isEmpty] #7 InputAdapter BroadcastExchange #18 WholeStageCodegen (25) @@ -175,8 +175,8 @@ TakeOrderedAndProject [c_first_name,c_last_name,sales] Project [c_customer_sk] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [tpcds_cmax] #1 - HashAggregate [c_customer_sk,isEmpty,sum] [isEmpty,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,isEmpty,sum] #7 + ReusedExchange [c_customer_sk,sum,isEmpty] #7 InputAdapter ReusedExchange [d_date_sk] #15 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index d1e4f3e242fc5..6e0a5ced1992a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -155,7 +155,7 @@ Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id= Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_zip), IsNotNull(ca_country)] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (25) ColumnarToRow @@ -163,7 +163,7 @@ Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (26) Filter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_zip#26) AND isnotnull(ca_country#27)) +Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (27) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_zip#23] @@ -176,11 +176,11 @@ Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, c (29) BroadcastExchange Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[4, string, true]), input[0, int, true]),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] (30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [c_birth_country#17, ss_store_sk#3] -Right keys [2]: [upper(ca_country#27), s_store_sk#19] +Left keys [2]: [ss_store_sk#3, c_birth_country#17] +Right keys [2]: [s_store_sk#19, upper(ca_country#27)] Join condition: None (31) Project [codegen id : 8] @@ -452,11 +452,11 @@ Input [16]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk (77) Exchange Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(s_zip#23, c_birth_country#17, 5), true, [id=#53] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] (78) Sort [codegen id : 11] Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [s_zip#23 ASC NULLS FIRST, c_birth_country#17 ASC NULLS FIRST], false, 0 +Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -474,15 +474,15 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(ca_zip#26, upper(ca_country#27), 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [ca_zip#26 ASC NULLS FIRST, upper(ca_country#27) ASC NULLS FIRST], false, 0 +Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 14] -Left keys [2]: [s_zip#23, c_birth_country#17] -Right keys [2]: [ca_zip#26, upper(ca_country#27)] +Left keys [2]: [c_birth_country#17, s_zip#23] +Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 6eb86a35357b0..f51d1972b630f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -1,64 +1,64 @@ WholeStageCodegen (14) - Project [c_first_name,c_last_name,paid,s_store_name] + Project [c_last_name,c_first_name,s_store_name,paid] Filter [sum(netpaid)] Subquery #1 WholeStageCodegen (20) - HashAggregate [count,sum] [(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),avg(netpaid),count,sum] + HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] InputAdapter Exchange #10 WholeStageCodegen (19) - HashAggregate [netpaid] [count,count,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #11 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 WholeStageCodegen (18) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (15) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #12 + Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [c_birth_country,ca_country,ca_zip,s_zip] + Project [s_store_name,s_state,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) Sort [c_birth_country,s_zip] InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [s_store_name,s_state,s_zip,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #15 WholeStageCodegen (2) - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid,ss_ticket_number] + Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid] BroadcastHashJoin [s_store_sk,ss_store_sk] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -68,17 +68,17 @@ WholeStageCodegen (14) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (9) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #18 WholeStageCodegen (8) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] @@ -88,92 +88,92 @@ WholeStageCodegen (14) Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] InputAdapter WholeStageCodegen (17) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #20 + Exchange [sr_ticket_number,sr_item_sk] #20 WholeStageCodegen (16) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - HashAggregate [c_first_name,c_last_name,isEmpty,s_store_name,sum] [isEmpty,paid,sum,sum(netpaid),sum(netpaid)] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #1 + Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (13) - HashAggregate [c_first_name,c_last_name,netpaid,s_store_name] [isEmpty,isEmpty,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #2 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 WholeStageCodegen (12) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (9) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_item_sk] #3 WholeStageCodegen (8) - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - BroadcastHashJoin [c_birth_country,ca_country,s_store_sk,ss_store_sk] - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,s_store_name,s_state,ca_state] + BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,ca_country] + Project [ss_item_sk,ss_store_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (2) - Project [i_color,i_current_price,i_manager_id,i_size,i_units,ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Filter [i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (5) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (4) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Project [ca_country,ca_state,s_state,s_store_name,s_store_sk] - BroadcastHashJoin [ca_zip,s_zip] + Project [s_store_sk,s_store_name,s_state,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] InputAdapter WholeStageCodegen (11) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #9 + Exchange [sr_ticket_number,sr_item_sk] #9 WholeStageCodegen (10) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index 88ea132e59e74..ac5580565d3f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -173,7 +173,7 @@ Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip Output [3]: [ca_state#27, ca_zip#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_zip), IsNotNull(ca_country)] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -181,7 +181,7 @@ Input [3]: [ca_state#27, ca_zip#28, ca_country#29] (31) Filter [codegen id : 5] Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Condition : (isnotnull(ca_zip#28) AND isnotnull(ca_country#29)) +Condition : (isnotnull(ca_country#29) AND isnotnull(ca_zip#28)) (32) BroadcastExchange Input [3]: [ca_state#27, ca_zip#28, ca_country#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 306d7f1f78e11..58b57dfaaef64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -1,125 +1,125 @@ WholeStageCodegen (8) - Project [c_first_name,c_last_name,paid,s_store_name] + Project [c_last_name,c_first_name,s_store_name,paid] Filter [sum(netpaid)] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),avg(netpaid),count,sum] + HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] InputAdapter Exchange #8 WholeStageCodegen (7) - HashAggregate [netpaid] [count,count,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #9 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #9 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - BroadcastHashJoin [c_birth_country,ca_country,ca_zip,s_zip] - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #10 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #11 WholeStageCodegen (2) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #12 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #13 WholeStageCodegen (4) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #14 WholeStageCodegen (5) Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] - HashAggregate [c_first_name,c_last_name,isEmpty,s_store_name,sum] [isEmpty,paid,sum,sum(netpaid),sum(netpaid)] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #1 + Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (7) - HashAggregate [c_first_name,c_last_name,netpaid,s_store_name] [isEmpty,isEmpty,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #2 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - BroadcastHashJoin [c_birth_country,ca_country,ca_zip,s_zip] - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 7d0932e3268fb..cbac3787cab6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -155,7 +155,7 @@ Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id= Output [3]: [ca_state#25, ca_zip#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_zip), IsNotNull(ca_country)] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (25) ColumnarToRow @@ -163,7 +163,7 @@ Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (26) Filter Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_zip#26) AND isnotnull(ca_country#27)) +Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (27) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_zip#23] @@ -176,11 +176,11 @@ Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, c (29) BroadcastExchange Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(upper(input[4, string, true]), input[0, int, true]),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] (30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [c_birth_country#17, ss_store_sk#3] -Right keys [2]: [upper(ca_country#27), s_store_sk#19] +Left keys [2]: [ss_store_sk#3, c_birth_country#17] +Right keys [2]: [s_store_sk#19, upper(ca_country#27)] Join condition: None (31) Project [codegen id : 8] @@ -452,11 +452,11 @@ Input [16]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_customer_sk (77) Exchange Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(s_zip#23, c_birth_country#17, 5), true, [id=#53] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] (78) Sort [codegen id : 11] Input [14]: [s_store_name#20, s_state#22, s_zip#23, ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [s_zip#23 ASC NULLS FIRST, c_birth_country#17 ASC NULLS FIRST], false, 0 +Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address Output [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -474,15 +474,15 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(ca_zip#26, upper(ca_country#27), 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [ca_zip#26 ASC NULLS FIRST, upper(ca_country#27) ASC NULLS FIRST], false, 0 +Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 14] -Left keys [2]: [s_zip#23, c_birth_country#17] -Right keys [2]: [ca_zip#26, upper(ca_country#27)] +Left keys [2]: [c_birth_country#17, s_zip#23] +Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 6eb86a35357b0..f51d1972b630f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -1,64 +1,64 @@ WholeStageCodegen (14) - Project [c_first_name,c_last_name,paid,s_store_name] + Project [c_last_name,c_first_name,s_store_name,paid] Filter [sum(netpaid)] Subquery #1 WholeStageCodegen (20) - HashAggregate [count,sum] [(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),avg(netpaid),count,sum] + HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] InputAdapter Exchange #10 WholeStageCodegen (19) - HashAggregate [netpaid] [count,count,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #11 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 WholeStageCodegen (18) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (15) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #12 + Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [c_birth_country,ca_country,ca_zip,s_zip] + Project [s_store_name,s_state,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) Sort [c_birth_country,s_zip] InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [s_store_name,s_state,s_zip,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (7) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #15 WholeStageCodegen (2) - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid,ss_ticket_number] + Project [s_store_name,s_state,s_zip,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid] BroadcastHashJoin [s_store_sk,ss_store_sk] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -68,17 +68,17 @@ WholeStageCodegen (14) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (9) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #18 WholeStageCodegen (8) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] @@ -88,92 +88,92 @@ WholeStageCodegen (14) Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] InputAdapter WholeStageCodegen (17) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #20 + Exchange [sr_ticket_number,sr_item_sk] #20 WholeStageCodegen (16) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - HashAggregate [c_first_name,c_last_name,isEmpty,s_store_name,sum] [isEmpty,paid,sum,sum(netpaid),sum(netpaid)] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #1 + Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (13) - HashAggregate [c_first_name,c_last_name,netpaid,s_store_name] [isEmpty,isEmpty,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #2 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 WholeStageCodegen (12) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (9) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_item_sk] #3 WholeStageCodegen (8) - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - BroadcastHashJoin [c_birth_country,ca_country,s_store_sk,ss_store_sk] - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,s_store_name,s_state,ca_state] + BroadcastHashJoin [ss_store_sk,c_birth_country,s_store_sk,ca_country] + Project [ss_item_sk,ss_store_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (2) - Project [i_color,i_current_price,i_manager_id,i_size,i_units,ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Filter [i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (5) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (4) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Project [ca_country,ca_state,s_state,s_store_name,s_store_sk] - BroadcastHashJoin [ca_zip,s_zip] + Project [s_store_sk,s_store_name,s_state,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] InputAdapter WholeStageCodegen (11) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #9 + Exchange [sr_ticket_number,sr_item_sk] #9 WholeStageCodegen (10) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 08fb812d3f0e5..0cf8c16a038ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -173,7 +173,7 @@ Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip Output [3]: [ca_state#27, ca_zip#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_zip), IsNotNull(ca_country)] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -181,7 +181,7 @@ Input [3]: [ca_state#27, ca_zip#28, ca_country#29] (31) Filter [codegen id : 5] Input [3]: [ca_state#27, ca_zip#28, ca_country#29] -Condition : (isnotnull(ca_zip#28) AND isnotnull(ca_country#29)) +Condition : (isnotnull(ca_country#29) AND isnotnull(ca_zip#28)) (32) BroadcastExchange Input [3]: [ca_state#27, ca_zip#28, ca_country#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 306d7f1f78e11..58b57dfaaef64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -1,125 +1,125 @@ WholeStageCodegen (8) - Project [c_first_name,c_last_name,paid,s_store_name] + Project [c_last_name,c_first_name,s_store_name,paid] Filter [sum(netpaid)] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),avg(netpaid),count,sum] + HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] InputAdapter Exchange #8 WholeStageCodegen (7) - HashAggregate [netpaid] [count,count,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #9 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #9 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - BroadcastHashJoin [c_birth_country,ca_country,ca_zip,s_zip] - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #10 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #11 WholeStageCodegen (2) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #12 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #13 WholeStageCodegen (4) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #14 WholeStageCodegen (5) Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] - HashAggregate [c_first_name,c_last_name,isEmpty,s_store_name,sum] [isEmpty,paid,sum,sum(netpaid),sum(netpaid)] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #1 + Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (7) - HashAggregate [c_first_name,c_last_name,netpaid,s_store_name] [isEmpty,isEmpty,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #2 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - BroadcastHashJoin [c_birth_country,ca_country,ca_zip,s_zip] - Project [c_birth_country,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) - Filter [c_birth_country,c_customer_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) Filter [ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index 1e703b42648ab..87a72d3bbe777 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -171,17 +171,17 @@ Input [9]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, (26) Exchange Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint), 5), true, [id=#20] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#20] (27) Sort [codegen id : 8] Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: [cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST], false, 0 +Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.date_dim Output [3]: [d_date_sk#21, d_year#22, d_moy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] @@ -189,7 +189,7 @@ Input [3]: [d_date_sk#21, d_year#22, d_moy#23] (30) Filter [codegen id : 9] Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_moy#23 >= 4)) AND (d_moy#23 <= 10)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#21)) +Condition : (((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 >= 4)) AND (d_moy#23 <= 10)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#21)) (31) Project [codegen id : 9] Output [1]: [d_date_sk#21] @@ -203,7 +203,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Output [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (34) ColumnarToRow @@ -211,7 +211,7 @@ Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_ (35) Filter Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Condition : (((isnotnull(sr_ticket_number#28) AND isnotnull(sr_customer_sk#27)) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_returned_date_sk#25)) +Condition : (((isnotnull(sr_customer_sk#27) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_ticket_number#28)) AND isnotnull(sr_returned_date_sk#25)) (36) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cast(d_date_sk#21 as bigint)] @@ -224,15 +224,15 @@ Input [6]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk# (38) Exchange Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Arguments: hashpartitioning(sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27, 5), true, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28, 5), true, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_net_loss#29] -Arguments: [sr_ticket_number#28 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_customer_sk#27 ASC NULLS FIRST], false, 0 +Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_ticket_number#28 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint)] -Right keys [3]: [sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28] Join condition: None (41) Project [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index af6cf2abe7d10..8e61cf9c519fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -1,57 +1,57 @@ -TakeOrderedAndProject [catalog_sales_profit,i_item_desc,i_item_id,s_store_id,s_store_name,store_returns_loss,store_sales_profit] +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] WholeStageCodegen (18) - HashAggregate [i_item_desc,i_item_id,s_store_id,s_store_name,sum,sum,sum] [catalog_sales_profit,store_returns_loss,store_sales_profit,sum,sum,sum,sum(UnscaledValue(cs_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] InputAdapter - Exchange [i_item_desc,i_item_id,s_store_id,s_store_name] #1 + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 WholeStageCodegen (17) - HashAggregate [cs_net_profit,i_item_desc,i_item_id,s_store_id,s_store_name,sr_net_loss,ss_net_profit] [sum,sum,sum,sum,sum,sum] - Project [cs_net_profit,i_item_desc,i_item_id,s_store_id,s_store_name,sr_net_loss,ss_net_profit] - SortMergeJoin [cs_bill_customer_sk,cs_item_sk,sr_customer_sk,sr_item_sk] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] + Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (13) Sort [sr_customer_sk,sr_item_sk] InputAdapter Exchange [sr_customer_sk,sr_item_sk] #2 WholeStageCodegen (12) - Project [i_item_desc,i_item_id,s_store_id,s_store_name,sr_customer_sk,sr_item_sk,sr_net_loss,ss_net_profit] - SortMergeJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] + Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (8) Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 WholeStageCodegen (7) - Project [i_item_desc,i_item_id,s_store_id,s_store_name,ss_customer_sk,ss_item_sk,ss_net_profit,ss_ticket_number] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (3) - Project [s_store_id,s_store_name,ss_customer_sk,ss_item_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] @@ -61,27 +61,27 @@ TakeOrderedAndProject [catalog_sales_profit,i_item_desc,i_item_id,s_store_id,s_s Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] InputAdapter WholeStageCodegen (11) Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) - Project [sr_customer_sk,sr_item_sk,sr_net_loss,sr_ticket_number] + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [d_date_sk,sr_returned_date_sk] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_net_loss,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] @@ -93,6 +93,6 @@ TakeOrderedAndProject [catalog_sales_profit,i_item_desc,i_item_id,s_store_id,s_s Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 8acd8414fb1d3..6bdd709a7d7a5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -53,7 +53,7 @@ TakeOrderedAndProject (48) Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_net_profit#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 8] @@ -61,13 +61,13 @@ Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss (3) Filter [codegen id : 8] Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_net_profit#6] -Condition : ((((isnotnull(ss_item_sk#2) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) +Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) (4) Scan parquet default.store_returns Output [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -75,7 +75,7 @@ Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_num (6) Filter [codegen id : 1] Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] -Condition : (((isnotnull(sr_ticket_number#10) AND isnotnull(sr_customer_sk#9)) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_returned_date_sk#7)) +Condition : (((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_returned_date_sk#7)) (7) BroadcastExchange Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 3de0f3d20ae08..4a40bdaff6da5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,58 +1,58 @@ -TakeOrderedAndProject [catalog_sales_profit,i_item_desc,i_item_id,s_store_id,s_store_name,store_returns_loss,store_sales_profit] +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] WholeStageCodegen (9) - HashAggregate [i_item_desc,i_item_id,s_store_id,s_store_name,sum,sum,sum] [catalog_sales_profit,store_returns_loss,store_sales_profit,sum,sum,sum,sum(UnscaledValue(cs_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] InputAdapter - Exchange [i_item_desc,i_item_id,s_store_id,s_store_name] #1 + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 WholeStageCodegen (8) - HashAggregate [cs_net_profit,i_item_desc,i_item_id,s_store_id,s_store_name,sr_net_loss,ss_net_profit] [sum,sum,sum,sum,sum,sum] - Project [cs_net_profit,i_item_desc,i_item_id,s_store_id,s_store_name,sr_net_loss,ss_net_profit] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [cs_net_profit,s_store_id,s_store_name,sr_net_loss,ss_item_sk,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [cs_net_profit,sr_net_loss,ss_item_sk,ss_net_profit,ss_store_sk] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] + Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_net_profit,cs_sold_date_sk,sr_net_loss,ss_item_sk,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [cs_net_profit,cs_sold_date_sk,sr_net_loss,sr_returned_date_sk,ss_item_sk,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [cs_net_profit,cs_sold_date_sk,sr_net_loss,sr_returned_date_sk,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,sr_customer_sk,sr_item_sk] - Project [sr_customer_sk,sr_item_sk,sr_net_loss,sr_returned_date_sk,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_sold_date_sk,cs_net_profit] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_returned_date_sk,sr_net_loss,cs_sold_date_sk,cs_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_net_profit,sr_returned_date_sk,sr_net_loss,cs_sold_date_sk,cs_net_profit] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_net_profit,sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_net_loss,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -61,11 +61,11 @@ TakeOrderedAndProject [catalog_sales_profit,i_item_desc,i_item_id,s_store_id,s_s Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt index b6917a75fe493..2a366c86fb264 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt @@ -53,7 +53,7 @@ Condition : (((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_sold_date_sk#1)) A Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_marital_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_education_status#12)) AND isnotnull(cd_marital_status#11)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/simplified.txt index 9ce1856692adb..d4ae376d5b425 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_sales_price)),avg(cast(cs_quantity as bigint)),count,count,count,count,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as bigint)),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) - HashAggregate [cs_coupon_amt,cs_list_price,cs_quantity,cs_sales_price,i_item_id] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_coupon_amt,cs_list_price,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_coupon_amt,cs_item_sk,cs_list_price,cs_quantity,cs_sales_price] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_coupon_amt,cs_item_sk,cs_list_price,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_coupon_amt,cs_item_sk,cs_list_price,cs_promo_sk,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_sold_date_sk,cs_item_sk,cs_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_promo_sk,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -32,12 +32,12 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [p_channel_email,p_channel_event,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_email,p_channel_event,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -47,4 +47,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index 2ad3da8e77a14..12e953427394f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -53,7 +53,7 @@ Condition : (((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_sold_date_sk#1)) A Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_marital_status#11) AND isnotnull(cd_gender#10)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index cd4705f97a303..94ce760c24952 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_sales_price)),avg(cast(cs_quantity as bigint)),count,count,count,count,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as bigint)),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) - HashAggregate [cs_coupon_amt,cs_list_price,cs_quantity,cs_sales_price,i_item_id] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_coupon_amt,cs_list_price,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_coupon_amt,cs_list_price,cs_promo_sk,cs_quantity,cs_sales_price,i_item_id] + Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_coupon_amt,cs_item_sk,cs_list_price,cs_promo_sk,cs_quantity,cs_sales_price] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_coupon_amt,cs_item_sk,cs_list_price,cs_promo_sk,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_sold_date_sk,cs_item_sk,cs_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_promo_sk,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -39,7 +39,7 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -47,4 +47,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [p_channel_email,p_channel_event,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_email,p_channel_event,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index 8fc22b44b7ca5..16aeff7a222ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -53,7 +53,7 @@ Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND is Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_marital_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_education_status#12)) AND isnotnull(cd_marital_status#11)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/simplified.txt index df5dc914d869b..204094d444927 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,s_state,spark_grouping_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_sales_price)),avg(cast(ss_quantity as bigint)),count,count,count,count,g_state,sum,sum,sum,sum] + HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state,spark_grouping_id] #1 WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Expand [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - Project [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -40,11 +40,11 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index b14b1847940b4..16aeff7a222ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -53,7 +53,7 @@ Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND is Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_marital_status#11) AND isnotnull(cd_gender#10)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index df5dc914d869b..204094d444927 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,s_state,spark_grouping_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_sales_price)),avg(cast(ss_quantity as bigint)),count,count,count,count,g_state,sum,sum,sum,sum] + HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state,spark_grouping_id] #1 WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Expand [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - Project [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -40,11 +40,11 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt index 7bd64fdc41260..d896002b0965d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/simplified.txt @@ -5,103 +5,103 @@ CollectLimit BroadcastNestedLoopJoin BroadcastNestedLoopJoin WholeStageCodegen (3) - HashAggregate [count,count,count,sum] [B1_CNT,B1_CNTD,B1_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] InputAdapter Exchange #1 WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #2 WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #3 WholeStageCodegen (6) - HashAggregate [count,count,count,sum] [B2_CNT,B2_CNTD,B2_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] InputAdapter Exchange #4 WholeStageCodegen (5) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #5 WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #6 WholeStageCodegen (9) - HashAggregate [count,count,count,sum] [B3_CNT,B3_CNTD,B3_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] InputAdapter Exchange #7 WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #8 WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #9 WholeStageCodegen (12) - HashAggregate [count,count,count,sum] [B4_CNT,B4_CNTD,B4_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] InputAdapter Exchange #10 WholeStageCodegen (11) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #11 WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #12 WholeStageCodegen (15) - HashAggregate [count,count,count,sum] [B5_CNT,B5_CNTD,B5_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] InputAdapter Exchange #13 WholeStageCodegen (14) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #14 WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #15 WholeStageCodegen (18) - HashAggregate [count,count,count,sum] [B6_CNT,B6_CNTD,B6_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] InputAdapter Exchange #16 WholeStageCodegen (17) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #17 WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt index 7bd64fdc41260..d896002b0965d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt @@ -5,103 +5,103 @@ CollectLimit BroadcastNestedLoopJoin BroadcastNestedLoopJoin WholeStageCodegen (3) - HashAggregate [count,count,count,sum] [B1_CNT,B1_CNTD,B1_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] InputAdapter Exchange #1 WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #2 WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #3 WholeStageCodegen (6) - HashAggregate [count,count,count,sum] [B2_CNT,B2_CNTD,B2_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] InputAdapter Exchange #4 WholeStageCodegen (5) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #5 WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #6 WholeStageCodegen (9) - HashAggregate [count,count,count,sum] [B3_CNT,B3_CNTD,B3_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] InputAdapter Exchange #7 WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #8 WholeStageCodegen (7) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #9 WholeStageCodegen (12) - HashAggregate [count,count,count,sum] [B4_CNT,B4_CNTD,B4_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] InputAdapter Exchange #10 WholeStageCodegen (11) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #11 WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #12 WholeStageCodegen (15) - HashAggregate [count,count,count,sum] [B5_CNT,B5_CNTD,B5_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] InputAdapter Exchange #13 WholeStageCodegen (14) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #14 WholeStageCodegen (13) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] BroadcastExchange #15 WholeStageCodegen (18) - HashAggregate [count,count,count,sum] [B6_CNT,B6_CNTD,B6_LP,avg(UnscaledValue(ss_list_price)),count,count,count,count(ss_list_price),count(ss_list_price),sum] + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] InputAdapter Exchange #16 WholeStageCodegen (17) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count,count,count(ss_list_price),count(ss_list_price),sum,sum] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] InputAdapter Exchange [ss_list_price] #17 WholeStageCodegen (16) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count,count,count,count,count(ss_list_price),sum,sum] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] Project [ss_list_price] - Filter [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Filter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_coupon_amt,ss_list_price,ss_quantity,ss_wholesale_cost] + Scan parquet default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index 31d05221a981c..35e24698c517e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -175,17 +175,17 @@ Input [9]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s (26) Exchange Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint), 5), true, [id=#20] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#20] (27) Sort [codegen id : 8] Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: [cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST], false, 0 +Arguments: [cast(ss_customer_sk#3 as bigint) ASC NULLS FIRST, cast(ss_item_sk#2 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#5 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.date_dim Output [3]: [d_date_sk#21, d_year#22, d_moy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] @@ -193,7 +193,7 @@ Input [3]: [d_date_sk#21, d_year#22, d_moy#23] (30) Filter [codegen id : 9] Input [3]: [d_date_sk#21, d_year#22, d_moy#23] -Condition : (((((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_moy#23 >= 9)) AND (d_moy#23 <= 12)) AND (d_year#22 = 1999)) AND isnotnull(d_date_sk#21)) +Condition : (((((isnotnull(d_moy#23) AND isnotnull(d_year#22)) AND (d_moy#23 >= 9)) AND (d_moy#23 <= 12)) AND (d_year#22 = 1999)) AND isnotnull(d_date_sk#21)) (31) Project [codegen id : 9] Output [1]: [d_date_sk#21] @@ -207,7 +207,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Output [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (34) ColumnarToRow @@ -215,7 +215,7 @@ Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_ (35) Filter Input [5]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Condition : (((isnotnull(sr_ticket_number#28) AND isnotnull(sr_customer_sk#27)) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_returned_date_sk#25)) +Condition : (((isnotnull(sr_customer_sk#27) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_ticket_number#28)) AND isnotnull(sr_returned_date_sk#25)) (36) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cast(d_date_sk#21 as bigint)] @@ -228,15 +228,15 @@ Input [6]: [d_date_sk#21, sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk# (38) Exchange Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Arguments: hashpartitioning(sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27, 5), true, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28, 5), true, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28, sr_return_quantity#29] -Arguments: [sr_ticket_number#28 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_customer_sk#27 ASC NULLS FIRST], false, 0 +Arguments: [sr_customer_sk#27 ASC NULLS FIRST, sr_item_sk#26 ASC NULLS FIRST, sr_ticket_number#28 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_ticket_number#5 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_customer_sk#3 as bigint)] -Right keys [3]: [sr_ticket_number#28, sr_item_sk#26, sr_customer_sk#27] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#27, sr_item_sk#26, sr_ticket_number#28] Join condition: None (41) Project [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt index 79ec8f0b8315d..f10b8e245c50e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/simplified.txt @@ -1,57 +1,57 @@ -TakeOrderedAndProject [catalog_sales_quantity,i_item_desc,i_item_id,s_store_id,s_store_name,store_returns_quantity,store_sales_quantity] +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] WholeStageCodegen (18) - HashAggregate [i_item_desc,i_item_id,s_store_id,s_store_name,sum,sum,sum] [catalog_sales_quantity,store_returns_quantity,store_sales_quantity,sum,sum,sum,sum(cast(cs_quantity as bigint)),sum(cast(sr_return_quantity as bigint)),sum(cast(ss_quantity as bigint))] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(cast(sr_return_quantity as bigint)),sum(cast(cs_quantity as bigint)),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] InputAdapter - Exchange [i_item_desc,i_item_id,s_store_id,s_store_name] #1 + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 WholeStageCodegen (17) - HashAggregate [cs_quantity,i_item_desc,i_item_id,s_store_id,s_store_name,sr_return_quantity,ss_quantity] [sum,sum,sum,sum,sum,sum] - Project [cs_quantity,i_item_desc,i_item_id,s_store_id,s_store_name,sr_return_quantity,ss_quantity] - SortMergeJoin [cs_bill_customer_sk,cs_item_sk,sr_customer_sk,sr_item_sk] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (13) Sort [sr_customer_sk,sr_item_sk] InputAdapter Exchange [sr_customer_sk,sr_item_sk] #2 WholeStageCodegen (12) - Project [i_item_desc,i_item_id,s_store_id,s_store_name,sr_customer_sk,sr_item_sk,sr_return_quantity,ss_quantity] - SortMergeJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] + Project [ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (8) Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 WholeStageCodegen (7) - Project [i_item_desc,i_item_id,s_store_id,s_store_name,ss_customer_sk,ss_item_sk,ss_quantity,ss_ticket_number] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (3) - Project [s_store_id,s_store_name,ss_customer_sk,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_quantity,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] @@ -61,27 +61,27 @@ TakeOrderedAndProject [catalog_sales_quantity,i_item_desc,i_item_id,s_store_id,s Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] InputAdapter WholeStageCodegen (11) Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 WholeStageCodegen (10) - Project [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_ticket_number] + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [d_date_sk,sr_returned_date_sk] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter WholeStageCodegen (16) Sort [cs_bill_customer_sk,cs_item_sk] @@ -93,12 +93,12 @@ TakeOrderedAndProject [catalog_sales_quantity,i_item_desc,i_item_id,s_store_id,s Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter BroadcastExchange #11 WholeStageCodegen (14) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index a45526688a30a..d1d99897da995 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -57,7 +57,7 @@ TakeOrderedAndProject (52) Output [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 8] @@ -65,13 +65,13 @@ Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss (3) Filter [codegen id : 8] Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_quantity#6] -Condition : ((((isnotnull(ss_item_sk#2) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) +Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_sold_date_sk#1)) AND isnotnull(ss_store_sk#4)) (4) Scan parquet default.store_returns Output [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -79,7 +79,7 @@ Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_num (6) Filter [codegen id : 1] Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] -Condition : (((isnotnull(sr_ticket_number#10) AND isnotnull(sr_customer_sk#9)) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_returned_date_sk#7)) +Condition : (((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) AND isnotnull(sr_returned_date_sk#7)) (7) BroadcastExchange Input [5]: [sr_returned_date_sk#7, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 26b7884072011..26e4e3a5591f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -1,63 +1,63 @@ -TakeOrderedAndProject [catalog_sales_quantity,i_item_desc,i_item_id,s_store_id,s_store_name,store_returns_quantity,store_sales_quantity] +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] WholeStageCodegen (9) - HashAggregate [i_item_desc,i_item_id,s_store_id,s_store_name,sum,sum,sum] [catalog_sales_quantity,store_returns_quantity,store_sales_quantity,sum,sum,sum,sum(cast(cs_quantity as bigint)),sum(cast(sr_return_quantity as bigint)),sum(cast(ss_quantity as bigint))] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(cast(sr_return_quantity as bigint)),sum(cast(cs_quantity as bigint)),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] InputAdapter - Exchange [i_item_desc,i_item_id,s_store_id,s_store_name] #1 + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 WholeStageCodegen (8) - HashAggregate [cs_quantity,i_item_desc,i_item_id,s_store_id,s_store_name,sr_return_quantity,ss_quantity] [sum,sum,sum,sum,sum,sum] - Project [cs_quantity,i_item_desc,i_item_id,s_store_id,s_store_name,sr_return_quantity,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [cs_quantity,s_store_id,s_store_name,sr_return_quantity,ss_item_sk,ss_quantity] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [cs_quantity,sr_return_quantity,ss_item_sk,ss_quantity,ss_store_sk] + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_sold_date_sk,sr_return_quantity,ss_item_sk,ss_quantity,ss_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [cs_quantity,cs_sold_date_sk,sr_return_quantity,sr_returned_date_sk,ss_item_sk,ss_quantity,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [cs_quantity,cs_sold_date_sk,sr_return_quantity,sr_returned_date_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,sr_customer_sk,sr_item_sk] - Project [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_returned_date_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_sold_date_sk,cs_quantity] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_return_quantity,cs_sold_date_sk,cs_quantity] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Filter [sr_customer_sk,sr_item_sk,sr_ticket_number,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_return_quantity,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -67,11 +67,11 @@ TakeOrderedAndProject [catalog_sales_quantity,i_item_desc,i_item_id,s_store_id,s Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/simplified.txt index f12df6c472196..3412ecfeea026 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,sum_agg] +TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,sum,sum(UnscaledValue(ss_ext_sales_price)),sum_agg] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_brand_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy] + Filter [d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index 4008c7b77cd8b..ed3a06904d46e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,sum_agg] +TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,sum,sum(UnscaledValue(ss_ext_sales_price)),sum_agg] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy] + Filter [d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt index d60728f540ce4..4a53f2266fb26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_customer_id,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation,ctr_total_return] +TakeOrderedAndProject [c_customer_id,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,ctr_total_return] WholeStageCodegen (17) - Project [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_customer_id,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation,ctr_total_return] - BroadcastHashJoin [c_customer_sk,ctr_customer_sk] + Project [c_customer_id,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,ctr_total_return] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,ctr_state,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] - HashAggregate [ca_state,sum,wr_returning_customer_sk] [ctr_customer_sk,ctr_state,ctr_total_return,sum,sum(UnscaledValue(wr_return_amt))] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,wr_returning_customer_sk] #1 + Exchange [wr_returning_customer_sk,ca_state] #1 WholeStageCodegen (6) - HashAggregate [ca_state,wr_return_amt,wr_returning_customer_sk] [sum,sum] - Project [ca_state,wr_return_amt,wr_returning_customer_sk] - SortMergeJoin [ca_address_sk,wr_returning_addr_sk] + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + SortMergeJoin [wr_returning_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (3) Sort [wr_returning_addr_sk] InputAdapter Exchange [wr_returning_addr_sk] #2 WholeStageCodegen (2) - Project [wr_return_amt,wr_returning_addr_sk,wr_returning_customer_sk] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_return_amt,wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -46,30 +46,30 @@ TakeOrderedAndProject [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_ BroadcastExchange #5 WholeStageCodegen (14) Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [count,ctr_state,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_state,sum] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] InputAdapter Exchange [ctr_state] #6 WholeStageCodegen (13) - HashAggregate [ctr_state,ctr_total_return] [count,count,sum,sum] - HashAggregate [ca_state,sum,wr_returning_customer_sk] [ctr_state,ctr_total_return,sum,sum(UnscaledValue(wr_return_amt))] + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,wr_returning_customer_sk] #7 + Exchange [wr_returning_customer_sk,ca_state] #7 WholeStageCodegen (12) - HashAggregate [ca_state,wr_return_amt,wr_returning_customer_sk] [sum,sum] - Project [ca_state,wr_return_amt,wr_returning_customer_sk] - SortMergeJoin [ca_address_sk,wr_returning_addr_sk] + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + SortMergeJoin [wr_returning_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (9) Sort [wr_returning_addr_sk] InputAdapter Exchange [wr_returning_addr_sk] #8 WholeStageCodegen (8) - Project [wr_return_amt,wr_returning_addr_sk,wr_returning_customer_sk] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_returned_date_sk,wr_returning_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_return_amt,wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -80,17 +80,17 @@ TakeOrderedAndProject [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_ InputAdapter BroadcastExchange #9 WholeStageCodegen (16) - Project [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation] + Project [c_customer_sk,c_customer_id,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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_current_addr_sk,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation] + Scan parquet 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 #10 WholeStageCodegen (15) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index afc390489e5aa..f9ea8ad58d339 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_customer_id,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation,ctr_total_return] +TakeOrderedAndProject [c_customer_id,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,ctr_total_return] WholeStageCodegen (11) - Project [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_customer_id,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation,ctr_total_return] + Project [c_customer_id,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,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_current_addr_sk,c_customer_id,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation,ctr_total_return] - BroadcastHashJoin [c_customer_sk,ctr_customer_sk] + Project [ctr_total_return,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] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,ctr_state,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] - HashAggregate [ca_state,sum,wr_returning_customer_sk] [ctr_customer_sk,ctr_state,ctr_total_return,sum,sum(UnscaledValue(wr_return_amt))] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,wr_returning_customer_sk] #1 + Exchange [wr_returning_customer_sk,ca_state] #1 WholeStageCodegen (3) - HashAggregate [ca_state,wr_return_amt,wr_returning_customer_sk] [sum,sum] - Project [ca_state,wr_return_amt,wr_returning_customer_sk] - BroadcastHashJoin [ca_address_sk,wr_returning_addr_sk] - Project [wr_return_amt,wr_returning_addr_sk,wr_returning_customer_sk] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_return_amt,wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -39,24 +39,24 @@ TakeOrderedAndProject [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_ BroadcastExchange #4 WholeStageCodegen (8) Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [count,ctr_state,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_state,sum] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] InputAdapter Exchange [ctr_state] #5 WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [count,count,sum,sum] - HashAggregate [ca_state,sum,wr_returning_customer_sk] [ctr_state,ctr_total_return,sum,sum(UnscaledValue(wr_return_amt))] + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,wr_returning_customer_sk] #6 + Exchange [wr_returning_customer_sk,ca_state] #6 WholeStageCodegen (6) - HashAggregate [ca_state,wr_return_amt,wr_returning_customer_sk] [sum,sum] - Project [ca_state,wr_return_amt,wr_returning_customer_sk] - BroadcastHashJoin [ca_address_sk,wr_returning_addr_sk] - Project [wr_return_amt,wr_returning_addr_sk,wr_returning_customer_sk] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_returned_date_sk,wr_returning_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_return_amt,wr_returned_date_sk,wr_returning_addr_sk,wr_returning_customer_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -64,15 +64,15 @@ TakeOrderedAndProject [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_ InputAdapter BroadcastExchange #7 WholeStageCodegen (9) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_birth_day,c_birth_month,c_birth_year,c_current_addr_sk,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_last_review_date,c_login,c_preferred_cust_flag,c_salutation] + Scan parquet 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 #8 WholeStageCodegen (10) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt index 6892a843d42d7..9ec06b597cb64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/simplified.txt @@ -3,38 +3,38 @@ WholeStageCodegen (43) InputAdapter Exchange [ca_county] #1 WholeStageCodegen (42) - Project [ca_county,d_year,store_sales,store_sales,store_sales,web_sales,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,store_sales,store_sales,store_sales,web_sales,web_sales,web_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales] + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + Project [store_sales,store_sales,ca_county,d_year,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,store_sales,store_sales] + Project [store_sales,ca_county,store_sales] BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [store_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #2 WholeStageCodegen (6) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ss_ext_sales_price] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (3) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #3 WholeStageCodegen (2) - Project [d_qoy,d_year,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_sold_date_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter WholeStageCodegen (5) Sort [ca_address_sk] @@ -48,32 +48,32 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #6 WholeStageCodegen (13) - HashAggregate [ca_county,d_qoy,d_year,sum] [store_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #7 WholeStageCodegen (12) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ss_ext_sales_price] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (9) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #8 WholeStageCodegen (8) - Project [d_qoy,d_year,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_sold_date_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter WholeStageCodegen (11) Sort [ca_address_sk] @@ -82,32 +82,32 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #10 WholeStageCodegen (20) - HashAggregate [ca_county,d_qoy,d_year,sum] [store_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #11 WholeStageCodegen (19) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ss_ext_sales_price] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (16) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #12 WholeStageCodegen (15) - Project [d_qoy,d_year,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_sold_date_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #13 WholeStageCodegen (14) - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter WholeStageCodegen (18) Sort [ca_address_sk] @@ -116,31 +116,31 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #14 WholeStageCodegen (41) - Project [ca_county,web_sales,web_sales,web_sales] + Project [web_sales,ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,web_sales,web_sales] + Project [web_sales,ca_county,web_sales] BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),web_sales] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #15 WholeStageCodegen (26) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ws_ext_sales_price] - SortMergeJoin [ca_address_sk,ws_bill_addr_sk] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ws_bill_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (23) Sort [ws_bill_addr_sk] InputAdapter Exchange [ws_bill_addr_sk] #16 WholeStageCodegen (22) - Project [d_qoy,d_year,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_sold_date_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #9 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 InputAdapter WholeStageCodegen (25) Sort [ca_address_sk] @@ -149,27 +149,27 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #17 WholeStageCodegen (33) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),web_sales] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #18 WholeStageCodegen (32) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ws_ext_sales_price] - SortMergeJoin [ca_address_sk,ws_bill_addr_sk] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ws_bill_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (29) Sort [ws_bill_addr_sk] InputAdapter Exchange [ws_bill_addr_sk] #19 WholeStageCodegen (28) - Project [d_qoy,d_year,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_sold_date_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #13 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 InputAdapter WholeStageCodegen (31) Sort [ca_address_sk] @@ -178,27 +178,27 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #20 WholeStageCodegen (40) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),web_sales] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #21 WholeStageCodegen (39) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ws_ext_sales_price] - SortMergeJoin [ca_address_sk,ws_bill_addr_sk] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + SortMergeJoin [ws_bill_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (36) Sort [ws_bill_addr_sk] InputAdapter Exchange [ws_bill_addr_sk] #22 WholeStageCodegen (35) - Project [d_qoy,d_year,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_sold_date_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 InputAdapter WholeStageCodegen (38) Sort [ca_address_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index c937700f0be16..5a5d61f9d9151 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -3,34 +3,34 @@ WholeStageCodegen (25) InputAdapter Exchange [ca_county] #1 WholeStageCodegen (24) - Project [ca_county,d_year,store_sales,store_sales,store_sales,web_sales,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,store_sales,store_sales,web_sales,web_sales] - Project [ca_county,ca_county,d_year,store_sales,store_sales,store_sales,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,store_sales,store_sales,web_sales,web_sales] + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [store_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #2 WholeStageCodegen (3) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ss_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [d_qoy,d_year,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -41,110 +41,110 @@ WholeStageCodegen (25) InputAdapter BroadcastExchange #5 WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [store_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #6 WholeStageCodegen (6) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ss_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [d_qoy,d_year,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #7 WholeStageCodegen (4) - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter BroadcastExchange #8 WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [store_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #9 WholeStageCodegen (10) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ss_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [d_qoy,d_year,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),web_sales] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #12 WholeStageCodegen (14) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ws_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [d_qoy,d_year,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter BroadcastExchange #13 WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),web_sales] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #14 WholeStageCodegen (18) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ws_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [d_qoy,d_year,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #7 + ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter BroadcastExchange #15 WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),web_sales] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #16 WholeStageCodegen (22) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ca_county,d_qoy,d_year,ws_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [d_qoy,d_year,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #10 + ReusedExchange [d_date_sk,d_year,d_qoy] #10 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index 9bdab6f9682c8..03fa18228b7c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -3,32 +3,32 @@ CollectLimit Project BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk] - BroadcastHashJoin [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),cs_ext_discount_amt,cs_item_sk,i_item_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] InputAdapter BroadcastExchange #1 WholeStageCodegen (4) - Project [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [i_item_sk,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [i_item_sk] - Filter [i_item_sk,i_manufact_id] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] - HashAggregate [count,cs_item_sk,sum] [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),avg(UnscaledValue(cs_ext_discount_amt)),count,cs_item_sk,sum] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),cs_item_sk,sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) - HashAggregate [cs_ext_discount_amt,cs_item_sk] [count,count,sum,sum] - Project [cs_ext_discount_amt,cs_item_sk] + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + Project [cs_item_sk,cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_discount_amt,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_discount_amt] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,10 +36,10 @@ CollectLimit Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] - Filter [cs_ext_discount_amt,cs_item_sk,cs_sold_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] + Filter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_discount_amt,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_discount_amt] InputAdapter ReusedExchange [d_date_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 997e9db1f9241..7e4b826ccde8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -3,18 +3,18 @@ CollectLimit Project BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk] - BroadcastHashJoin [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),cs_ext_discount_amt,cs_item_sk,i_item_sk] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] + Project [cs_sold_date_sk,cs_ext_discount_amt,i_item_sk] BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_ext_discount_amt,cs_item_sk,cs_sold_date_sk] + Filter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_discount_amt,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_discount_amt] InputAdapter BroadcastExchange #1 WholeStageCodegen (1) Project [i_item_sk] - Filter [i_item_sk,i_manufact_id] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -22,17 +22,17 @@ CollectLimit BroadcastExchange #2 WholeStageCodegen (4) Filter [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6)))] - HashAggregate [count,cs_item_sk,sum] [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),avg(UnscaledValue(cs_ext_discount_amt)),count,cs_item_sk,sum] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(cs_ext_discount_amt) AS DECIMAL(11,6))),cs_item_sk,sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) - HashAggregate [cs_ext_discount_amt,cs_item_sk] [count,count,sum,sum] - Project [cs_ext_discount_amt,cs_item_sk] + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + Project [cs_item_sk,cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_discount_amt,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_discount_amt] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -40,6 +40,6 @@ CollectLimit Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index e810c4bfbf62d..410def2466e1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [i_manufact_id,total_sales] +TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (20) - HashAggregate [i_manufact_id,isEmpty,sum] [isEmpty,sum,sum(total_sales),total_sales] + HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_manufact_id] #1 WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total_sales] + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #2 WholeStageCodegen (5) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [i_manufact_id,ss_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [i_manufact_id,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -51,26 +51,26 @@ TakeOrderedAndProject [i_manufact_id,total_sales] BroadcastExchange #6 WholeStageCodegen (4) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum,sum(UnscaledValue(cs_ext_sales_price)),total_sales] + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #7 WholeStageCodegen (11) - HashAggregate [cs_ext_sales_price,i_manufact_id] [sum,sum] + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ca_address_sk,cs_bill_addr_sk] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -78,21 +78,21 @@ TakeOrderedAndProject [i_manufact_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #6 WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),total_sales] + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #8 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [i_manufact_id,ws_ext_sales_price] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [i_manufact_id,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 5b0cca34b3c70..14787f0bbce7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,41 +1,41 @@ -TakeOrderedAndProject [i_manufact_id,total_sales] +TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (20) - HashAggregate [i_manufact_id,isEmpty,sum] [isEmpty,sum,sum(total_sales),total_sales] + HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_manufact_id] #1 WholeStageCodegen (19) - HashAggregate [i_manufact_id,total_sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [i_manufact_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total_sales] + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #2 WholeStageCodegen (5) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [i_manufact_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] @@ -56,21 +56,21 @@ TakeOrderedAndProject [i_manufact_id,total_sales] InputAdapter Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (12) - HashAggregate [i_manufact_id,sum] [sum,sum(UnscaledValue(cs_ext_sales_price)),total_sales] + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #7 WholeStageCodegen (11) - HashAggregate [cs_ext_sales_price,i_manufact_id] [sum,sum] + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk] - BroadcastHashJoin [ca_address_sk,cs_bill_addr_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -78,21 +78,21 @@ TakeOrderedAndProject [i_manufact_id,total_sales] InputAdapter ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) - HashAggregate [i_manufact_id,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),total_sales] + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #8 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [i_manufact_id,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt index 94ec6ce18503e..d9b416ddba9ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (10) - Sort [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter - Exchange [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] #1 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 WholeStageCodegen (9) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] @@ -12,29 +12,29 @@ WholeStageCodegen (10) Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_customer_sk] #3 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -42,15 +42,15 @@ WholeStageCodegen (10) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] @@ -60,4 +60,4 @@ WholeStageCodegen (10) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 12cd87e119622..5af07f1d4ddef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -1,34 +1,34 @@ WholeStageCodegen (7) - Sort [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter - Exchange [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] #1 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 WholeStageCodegen (6) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk] #2 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,19 +36,19 @@ WholeStageCodegen (7) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt index 35dc74b2ced18..7c40dfcf8133a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/simplified.txt @@ -1,11 +1,11 @@ -TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),ca_state,cd_dep_college_count,cd_dep_employed_count,cd_gender,cd_marital_status,cnt1,cnt2,cnt3,max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_employed_count),min(cd_dep_college_count),min(cd_dep_count),min(cd_dep_employed_count)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] WholeStageCodegen (21) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,count,count,count,count,max,max,max,min,min,min,sum,sum,sum] [aggOrder,avg(cast(cd_dep_college_count as bigint)),avg(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),cnt1,cnt2,cnt3,count,count,count,count,count(1),max,max,max,max(cd_dep_college_count),max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_count),max(cd_dep_employed_count),max(cd_dep_employed_count),min,min,min,min(cd_dep_college_count),min(cd_dep_college_count),min(cd_dep_count),min(cd_dep_count),min(cd_dep_employed_count),min(cd_dep_employed_count),sum,sum,sum] + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cast(cd_dep_count as bigint)),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cast(cd_dep_employed_count as bigint)),min(cd_dep_college_count),max(cd_dep_college_count),avg(cast(cd_dep_college_count as bigint)),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),aggOrder,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter - Exchange [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] #1 + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (20) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] [count,count,count,count,count,count,count,count,max,max,max,max,max,max,min,min,min,min,min,min,sum,sum,sum,sum,sum,sum] - Project [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (17) @@ -21,7 +21,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( InputAdapter Exchange [c_current_addr_sk] #3 WholeStageCodegen (12) - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] @@ -35,37 +35,37 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #5 WholeStageCodegen (4) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_year,d_qoy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] WholeStageCodegen (8) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #7 WholeStageCodegen (7) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (11) @@ -78,7 +78,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -100,4 +100,4 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index 4f500e2b32b31..9f27114f7e0f8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),ca_state,cd_dep_college_count,cd_dep_employed_count,cd_gender,cd_marital_status,cnt1,cnt2,cnt3,max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_employed_count),min(cd_dep_college_count),min(cd_dep_count),min(cd_dep_employed_count)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,aggOrder,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] WholeStageCodegen (10) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,count,count,count,count,max,max,max,min,min,min,sum,sum,sum] [aggOrder,avg(cast(cd_dep_college_count as bigint)),avg(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),cnt1,cnt2,cnt3,count,count,count,count,count(1),max,max,max,max(cd_dep_college_count),max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_count),max(cd_dep_employed_count),max(cd_dep_employed_count),min,min,min,min(cd_dep_college_count),min(cd_dep_college_count),min(cd_dep_count),min(cd_dep_count),min(cd_dep_employed_count),min(cd_dep_employed_count),sum,sum,sum] + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cast(cd_dep_count as bigint)),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cast(cd_dep_employed_count as bigint)),min(cd_dep_college_count),max(cd_dep_college_count),avg(cast(cd_dep_college_count as bigint)),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),aggOrder,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter - Exchange [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] #1 + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (9) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] [count,count,count,count,count,count,count,count,max,max,max,max,max,max,min,min,min,min,min,min,sum,sum,sum,sum,sum,sum] - Project [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk,ca_state] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] @@ -17,33 +17,33 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_year,d_qoy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -54,7 +54,7 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -70,4 +70,4 @@ TakeOrderedAndProject [aggOrder,avg(cd_dep_college_count),avg(cd_dep_count),avg( Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/simplified.txt index bef14cb3851ac..208e0722524a4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] WholeStageCodegen (7) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w1,_w2,_w3] + Window [_w3,_w1,_w2] WholeStageCodegen (6) Sort [_w1,_w2,_w3] InputAdapter Exchange [_w1,_w2] #1 WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [_w1,_w2,_w3,gross_margin,lochierarchy,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w1,_w2,_w3,sum,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Expand [i_category,i_class,ss_ext_sales_price,ss_net_profit] - Project [i_category,i_class,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -39,11 +39,11 @@ TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_ Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index aa668141fd783..d4a0814528276 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] WholeStageCodegen (7) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w1,_w2,_w3] + Window [_w3,_w1,_w2] WholeStageCodegen (6) Sort [_w1,_w2,_w3] InputAdapter Exchange [_w1,_w2] #1 WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [_w1,_w2,_w3,gross_margin,lochierarchy,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w1,_w2,_w3,sum,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Expand [i_category,i_class,ss_ext_sales_price,ss_net_profit] - Project [i_category,i_class,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_category,i_class,ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -38,7 +38,7 @@ TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_ Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -46,4 +46,4 @@ TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_ Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/simplified.txt index aa3b9e817c910..0fc9d748fa1b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] WholeStageCodegen (8) - HashAggregate [i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter - Exchange [i_current_price,i_item_desc,i_item_id] #1 + Exchange [i_item_id,i_item_desc,i_current_price] #1 WholeStageCodegen (7) - HashAggregate [i_current_price,i_item_desc,i_item_id] - Project [i_current_price,i_item_desc,i_item_id] - SortMergeJoin [cs_item_sk,i_item_sk] + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + SortMergeJoin [i_item_sk,cs_item_sk] InputAdapter WholeStageCodegen (4) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #2 WholeStageCodegen (3) - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk,inv_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - Filter [i_current_price,i_item_sk,i_manufact_id] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + Filter [i_current_price,i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_desc,i_item_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] Project [inv_date_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + Filter [inv_quantity_on_hand,inv_item_sk,inv_date_sk] ColumnarToRow InputAdapter Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] @@ -37,7 +37,7 @@ TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (6) Sort [cs_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index f895fe34d0b11..6d3216fffedee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -1,26 +1,26 @@ -TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] WholeStageCodegen (5) - HashAggregate [i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter - Exchange [i_current_price,i_item_desc,i_item_id] #1 + Exchange [i_item_id,i_item_desc,i_current_price] #1 WholeStageCodegen (4) - HashAggregate [i_current_price,i_item_desc,i_item_id] - Project [i_current_price,i_item_desc,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk,inv_date_sk] + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - Filter [i_current_price,i_item_sk,i_manufact_id] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + Filter [i_current_price,i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_desc,i_item_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [inv_date_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + Filter [inv_quantity_on_hand,inv_item_sk,inv_date_sk] ColumnarToRow InputAdapter Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] @@ -31,7 +31,7 @@ TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt index 4a3f651d416fc..5bcd7dbb93022 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/simplified.txt @@ -1,49 +1,49 @@ CollectLimit WholeStageCodegen (26) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (25) HashAggregate [count,count] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #2 + Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (24) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - SortMergeJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] - SortMergeJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] WholeStageCodegen (7) - Sort [c_first_name,c_last_name,d_date] + Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #3 + Exchange [c_last_name,c_first_name,d_date] #3 WholeStageCodegen (6) - Project [c_first_name,c_last_name,d_date] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (2) - Project [d_date,ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (5) Sort [c_customer_sk] @@ -55,17 +55,17 @@ CollectLimit InputAdapter Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] WholeStageCodegen (15) - Sort [c_first_name,c_last_name,d_date] + Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #7 + Exchange [c_last_name,c_first_name,d_date] #7 WholeStageCodegen (14) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #8 + Exchange [c_last_name,c_first_name,d_date] #8 WholeStageCodegen (13) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (10) Sort [cs_bill_customer_sk] @@ -74,43 +74,43 @@ CollectLimit WholeStageCodegen (9) Project [cs_bill_customer_sk,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #5 + ReusedExchange [d_date_sk,d_date] #5 InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 WholeStageCodegen (23) - Sort [c_first_name,c_last_name,d_date] + Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #10 + Exchange [c_last_name,c_first_name,d_date] #10 WholeStageCodegen (22) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #11 + Exchange [c_last_name,c_first_name,d_date] #11 WholeStageCodegen (21) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (18) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #12 WholeStageCodegen (17) - Project [d_date,ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #5 + ReusedExchange [d_date_sk,d_date] #5 InputAdapter WholeStageCodegen (20) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index ea2c2fce9f4e1..10a2166ce761d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -1,37 +1,37 @@ CollectLimit WholeStageCodegen (13) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (12) HashAggregate [count,count] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #2 + Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (11) - HashAggregate [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] - BroadcastHashJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] - Project [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [d_date,ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + HashAggregate [c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -42,40 +42,40 @@ CollectLimit InputAdapter BroadcastExchange #5 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #6 + Exchange [c_last_name,c_first_name,d_date] #6 WholeStageCodegen (5) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] Project [cs_bill_customer_sk,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #3 + ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter BroadcastExchange #7 WholeStageCodegen (10) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #8 + Exchange [c_last_name,c_first_name,d_date] #8 WholeStageCodegen (9) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Project [d_date,ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #3 + ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt index c38a3b410e120..c0f519d2ec83b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt @@ -1,40 +1,40 @@ WholeStageCodegen (14) - Sort [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter - Exchange [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] #1 + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 WholeStageCodegen (13) - SortMergeJoin [i_item_sk,i_item_sk,w_warehouse_sk,w_warehouse_sk] + SortMergeJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] InputAdapter WholeStageCodegen (6) Sort [i_item_sk,w_warehouse_sk] InputAdapter Exchange [i_item_sk,w_warehouse_sk] #2 WholeStageCodegen (5) - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #3 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 WholeStageCodegen (4) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_moy,i_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [d_moy,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,d_moy,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -48,39 +48,39 @@ WholeStageCodegen (14) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter WholeStageCodegen (12) Sort [i_item_sk,w_warehouse_sk] InputAdapter Exchange [i_item_sk,w_warehouse_sk] #7 WholeStageCodegen (11) - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #8 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (10) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_moy,i_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [d_moy,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,d_moy,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter - ReusedExchange [w_warehouse_name,w_warehouse_sk] #6 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index c1a1b4dab7de7..f4e23c837b63a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -1,26 +1,26 @@ WholeStageCodegen (11) - Sort [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter - Exchange [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] #1 + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,i_item_sk,w_warehouse_sk,w_warehouse_sk] - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #2 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -34,44 +34,44 @@ WholeStageCodegen (11) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (9) - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #7 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 WholeStageCodegen (8) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter ReusedExchange [i_item_sk] #3 InputAdapter - ReusedExchange [w_warehouse_name,w_warehouse_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #4 InputAdapter BroadcastExchange #8 WholeStageCodegen (7) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt index c38a3b410e120..c0f519d2ec83b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt @@ -1,40 +1,40 @@ WholeStageCodegen (14) - Sort [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter - Exchange [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] #1 + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 WholeStageCodegen (13) - SortMergeJoin [i_item_sk,i_item_sk,w_warehouse_sk,w_warehouse_sk] + SortMergeJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] InputAdapter WholeStageCodegen (6) Sort [i_item_sk,w_warehouse_sk] InputAdapter Exchange [i_item_sk,w_warehouse_sk] #2 WholeStageCodegen (5) - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #3 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 WholeStageCodegen (4) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_moy,i_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [d_moy,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,d_moy,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -48,39 +48,39 @@ WholeStageCodegen (14) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter WholeStageCodegen (12) Sort [i_item_sk,w_warehouse_sk] InputAdapter Exchange [i_item_sk,w_warehouse_sk] #7 WholeStageCodegen (11) - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #8 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (10) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [d_moy,i_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [d_moy,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,d_moy,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter - ReusedExchange [w_warehouse_name,w_warehouse_sk] #6 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index c1a1b4dab7de7..f4e23c837b63a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -1,26 +1,26 @@ WholeStageCodegen (11) - Sort [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter - Exchange [cov,cov,d_moy,d_moy,i_item_sk,mean,mean,w_warehouse_sk] #1 + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,i_item_sk,w_warehouse_sk,w_warehouse_sk] - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #2 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -34,44 +34,44 @@ WholeStageCodegen (11) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (9) - Project [d_moy,i_item_sk,mean,stdev,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [avg,count,d_moy,i_item_sk,m2,n,sum,w_warehouse_name,w_warehouse_sk] [avg,avg(cast(inv_quantity_on_hand as bigint)),count,m2,mean,n,stddev_samp(cast(inv_quantity_on_hand as double)),stdev,sum] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [d_moy,i_item_sk,w_warehouse_name,w_warehouse_sk] #7 + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 WholeStageCodegen (8) - HashAggregate [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] [avg,avg,count,count,m2,m2,n,n,sum,sum] - Project [d_moy,i_item_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,w_warehouse_name,w_warehouse_sk] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_date_sk,inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [i_item_sk,inv_date_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] + Project [inv_date_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter ReusedExchange [i_item_sk] #3 InputAdapter - ReusedExchange [w_warehouse_name,w_warehouse_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #4 InputAdapter BroadcastExchange #8 WholeStageCodegen (7) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt index a3798d59135dd..1a497cb6c89c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt @@ -1,18 +1,18 @@ -TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag] +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (53) - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag] + Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (44) - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (35) - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) @@ -24,29 +24,29 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi Exchange [customer_id] #1 WholeStageCodegen (7) Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #2 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #2 WholeStageCodegen (6) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #3 WholeStageCodegen (2) - Project [d_year,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -56,39 +56,39 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (4) - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter WholeStageCodegen (16) Sort [customer_id] InputAdapter Exchange [customer_id] #6 WholeStageCodegen (15) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #7 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #7 WholeStageCodegen (14) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (10) - Project [d_year,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -96,7 +96,7 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi WholeStageCodegen (13) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (25) Sort [customer_id] @@ -105,64 +105,64 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi WholeStageCodegen (24) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #11 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (23) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (20) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #12 WholeStageCodegen (19) - Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] + Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #14 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (32) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (29) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #15 WholeStageCodegen (28) - Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] + Project [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #9 InputAdapter WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (43) Sort [customer_id] @@ -171,61 +171,61 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi WholeStageCodegen (42) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #17 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 WholeStageCodegen (41) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (38) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #18 WholeStageCodegen (37) - Project [d_year,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (40) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (52) Sort [customer_id] InputAdapter Exchange [customer_id] #19 WholeStageCodegen (51) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #20 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 WholeStageCodegen (50) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (47) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #21 WholeStageCodegen (46) - Project [d_year,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #9 InputAdapter WholeStageCodegen (49) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 3d45f95afa9f0..017b34451ef7e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -1,64 +1,64 @@ -TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag] +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] WholeStageCodegen (24) - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag] + Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #1 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 WholeStageCodegen (3) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_birth_country,customer_email_address,customer_first_name,customer_id,customer_last_name,customer_login,customer_preferred_cust_flag,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #5 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 WholeStageCodegen (6) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -67,46 +67,46 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi WholeStageCodegen (11) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #8 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8 WholeStageCodegen (10) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,cs_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_sold_date_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) Filter [cs_bill_customer_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #10 WholeStageCodegen (15) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #11 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (14) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,d_year] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,cs_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_sold_date_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_list_price,cs_ext_sales_price,cs_ext_wholesale_cost,cs_sold_date_sk] #9 + ReusedExchange [cs_sold_date_sk,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #6 InputAdapter @@ -114,45 +114,45 @@ TakeOrderedAndProject [customer_birth_country,customer_email_address,customer_fi WholeStageCodegen (19) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #13 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #13 WholeStageCodegen (18) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost,ws_sold_date_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #14 WholeStageCodegen (16) Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #15 WholeStageCodegen (23) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,isEmpty,sum] [customer_id,isEmpty,sum,sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),year_total] + HashAggregate [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] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #16 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #16 WholeStageCodegen (22) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] [isEmpty,isEmpty,sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost,ws_sold_date_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_ext_sales_price,ws_ext_wholesale_cost,ws_sold_date_sk] #14 + ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price] #14 InputAdapter ReusedExchange [d_date_sk,d_year] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt index 243a3b5957787..6e976ada38863 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt @@ -1,56 +1,56 @@ -TakeOrderedAndProject [i_item_id,sales_after,sales_before,w_state] +TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (9) - HashAggregate [i_item_id,isEmpty,isEmpty,sum,sum,w_state] [isEmpty,isEmpty,sales_after,sales_before,sum,sum,sum(CASE WHEN (d_date < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sum(CASE WHEN (d_date >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)] + HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sum(CASE WHEN (d_date >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [i_item_id,w_state] #1 + Exchange [w_state,i_item_id] #1 WholeStageCodegen (8) - HashAggregate [cr_refunded_cash,cs_sales_price,d_date,i_item_id,w_state] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum] - Project [cr_refunded_cash,cs_sales_price,d_date,i_item_id,w_state] + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cr_refunded_cash,cs_sales_price,cs_warehouse_sk,d_date,i_item_id] + Project [cs_warehouse_sk,cs_sales_price,cr_refunded_cash,i_item_id,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_refunded_cash,cs_sales_price,cs_sold_date_sk,cs_warehouse_sk,i_item_id] + Project [cs_sold_date_sk,cs_warehouse_sk,cs_sales_price,cr_refunded_cash,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cr_refunded_cash,cs_item_sk,cs_sales_price,cs_sold_date_sk,cs_warehouse_sk] + Project [cs_sold_date_sk,cs_warehouse_sk,cs_item_sk,cs_sales_price,cr_refunded_cash] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (2) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #2 + Exchange [cs_order_number,cs_item_sk] #2 WholeStageCodegen (1) - Filter [cs_item_sk,cs_sold_date_sk,cs_warehouse_sk] + Filter [cs_warehouse_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price] WholeStageCodegen (4) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #3 + Exchange [cr_order_number,cr_item_sk] #3 WholeStageCodegen (3) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash] InputAdapter BroadcastExchange #4 WholeStageCodegen (5) - Project [i_item_id,i_item_sk] + Project [i_item_sk,i_item_id] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_current_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (6) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (7) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_state,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt index 067e778838b6d..b1e0f1f17dd0a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt @@ -1,26 +1,26 @@ -TakeOrderedAndProject [i_item_id,sales_after,sales_before,w_state] +TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] WholeStageCodegen (6) - HashAggregate [i_item_id,isEmpty,isEmpty,sum,sum,w_state] [isEmpty,isEmpty,sales_after,sales_before,sum,sum,sum(CASE WHEN (d_date < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sum(CASE WHEN (d_date >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END)] + HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sum(CASE WHEN (d_date >= 11027) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [i_item_id,w_state] #1 + Exchange [w_state,i_item_id] #1 WholeStageCodegen (5) - HashAggregate [cr_refunded_cash,cs_sales_price,d_date,i_item_id,w_state] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum] - Project [cr_refunded_cash,cs_sales_price,d_date,i_item_id,w_state] + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_refunded_cash,cs_sales_price,cs_sold_date_sk,i_item_id,w_state] + Project [cs_sold_date_sk,cs_sales_price,cr_refunded_cash,w_state,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cr_refunded_cash,cs_item_sk,cs_sales_price,cs_sold_date_sk,w_state] + Project [cs_sold_date_sk,cs_item_sk,cs_sales_price,cr_refunded_cash,w_state] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cr_refunded_cash,cs_item_sk,cs_sales_price,cs_sold_date_sk,cs_warehouse_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Filter [cs_item_sk,cs_sold_date_sk,cs_warehouse_sk] + Project [cs_sold_date_sk,cs_warehouse_sk,cs_item_sk,cs_sales_price,cr_refunded_cash] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Filter [cs_warehouse_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash] @@ -30,19 +30,19 @@ TakeOrderedAndProject [i_item_id,sales_after,sales_before,w_state] Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_state,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_state] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [i_item_id,i_item_sk] + Project [i_item_sk,i_item_id] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_current_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index c5e1d6cb8dce8..350aa9a3c572b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -8,22 +8,22 @@ TakeOrderedAndProject [i_product_name] Project [i_product_name] BroadcastHashJoin [i_manufact,i_manufact] Project [i_manufact,i_product_name] - Filter [i_manufact,i_manufact_id] + Filter [i_manufact_id,i_manufact] ColumnarToRow InputAdapter - Scan parquet default.item [i_manufact,i_manufact_id,i_product_name] + Scan parquet default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [i_manufact] Filter [alwaysTrue,item_cnt] - HashAggregate [count,i_manufact] [alwaysTrue,count,count(1),i_manufact,item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,alwaysTrue,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) HashAggregate [i_manufact] [count,count] Project [i_manufact] - Filter [i_category,i_color,i_manufact,i_size,i_units] + Filter [i_category,i_color,i_units,i_size,i_manufact] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_color,i_manufact,i_size,i_units] + Scan parquet default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index c5e1d6cb8dce8..350aa9a3c572b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -8,22 +8,22 @@ TakeOrderedAndProject [i_product_name] Project [i_product_name] BroadcastHashJoin [i_manufact,i_manufact] Project [i_manufact,i_product_name] - Filter [i_manufact,i_manufact_id] + Filter [i_manufact_id,i_manufact] ColumnarToRow InputAdapter - Scan parquet default.item [i_manufact,i_manufact_id,i_product_name] + Scan parquet default.item [i_manufact_id,i_manufact,i_product_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [i_manufact] Filter [alwaysTrue,item_cnt] - HashAggregate [count,i_manufact] [alwaysTrue,count,count(1),i_manufact,item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,alwaysTrue,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) HashAggregate [i_manufact] [count,count] Project [i_manufact] - Filter [i_category,i_color,i_manufact,i_size,i_units] + Filter [i_category,i_color,i_units,i_size,i_manufact] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_color,i_manufact,i_size,i_units] + Scan parquet default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/simplified.txt index 83050b9af3411..1ad7df6bb984a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [d_year,i_category,i_category_id,sum(ss_ext_sales_price)] +TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (4) - HashAggregate [d_year,i_category,i_category_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price)] + HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] InputAdapter - Exchange [d_year,i_category,i_category_id] #1 + Exchange [d_year,i_category_id,i_category] #1 WholeStageCodegen (3) - HashAggregate [d_year,i_category,i_category_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_category,i_category_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_category,i_category_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_category,i_category_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_category_id,i_category] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_category_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_category_id,i_category,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index c8fc6e5c63919..d9bb6de204184 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [d_year,i_category,i_category_id,sum(ss_ext_sales_price)] +TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] WholeStageCodegen (4) - HashAggregate [d_year,i_category,i_category_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price)] + HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] InputAdapter - Exchange [d_year,i_category,i_category_id] #1 + Exchange [d_year,i_category_id,i_category] #1 WholeStageCodegen (3) - HashAggregate [d_year,i_category,i_category_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_category,i_category_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_category,i_category_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_category_id,i_category] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_category_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/simplified.txt index 2ed0f8f445720..a366ea3d4696a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] +TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] WholeStageCodegen (4) - HashAggregate [s_store_id,s_store_name,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter - Exchange [s_store_id,s_store_name] #1 + Exchange [s_store_name,s_store_id] #1 WholeStageCodegen (3) - HashAggregate [d_day_name,s_store_id,s_store_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,s_store_id,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_day_name,ss_sales_price,ss_store_sk] + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk,d_day_name] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_day_name] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [s_store_id,s_store_name,s_store_sk] + Project [s_store_sk,s_store_id,s_store_name] Filter [s_gmt_offset,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_gmt_offset,s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index bf8ac2624db70..d2cb50bd5a41d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] +TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] WholeStageCodegen (4) - HashAggregate [s_store_id,s_store_name,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter - Exchange [s_store_id,s_store_name] #1 + Exchange [s_store_name,s_store_id] #1 WholeStageCodegen (3) - HashAggregate [d_day_name,s_store_id,s_store_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,s_store_id,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_day_name,ss_sales_price,ss_store_sk] + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_day_name] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_day_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [s_store_id,s_store_name,s_store_sk] + Project [s_store_sk,s_store_id,s_store_name] Filter [s_gmt_offset,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_gmt_offset,s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt index 96e07ea3d8a33..f2106ad7aadd3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt @@ -1,13 +1,13 @@ -TakeOrderedAndProject [best_performing,rnk,worst_performing] +TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (10) - Project [i_product_name,i_product_name,rnk] - BroadcastHashJoin [i_item_sk,item_sk] - Project [i_product_name,item_sk,rnk] - BroadcastHashJoin [i_item_sk,item_sk] - Project [item_sk,item_sk,rnk] + Project [rnk,i_product_name,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [rnk,item_sk,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [item_sk,rnk,item_sk] BroadcastHashJoin [rnk,rnk] Project [item_sk,rnk] - Filter [item_sk,rnk] + Filter [rnk,item_sk] InputAdapter Window [rank_col] WholeStageCodegen (3) @@ -19,31 +19,31 @@ TakeOrderedAndProject [best_performing,rnk,worst_performing] Filter [avg(ss_net_profit)] Subquery #1 WholeStageCodegen (2) - HashAggregate [count,ss_store_sk,sum] [avg(UnscaledValue(ss_net_profit)),count,rank_col,sum] + HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] InputAdapter Exchange [ss_store_sk] #3 WholeStageCodegen (1) - HashAggregate [ss_net_profit,ss_store_sk] [count,count,sum,sum] - Project [ss_net_profit,ss_store_sk] - Filter [ss_addr_sk,ss_store_sk] + HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] + Project [ss_store_sk,ss_net_profit] + Filter [ss_store_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_net_profit,ss_store_sk] - HashAggregate [count,ss_item_sk,sum] [avg(UnscaledValue(ss_net_profit)),avg(ss_net_profit),count,item_sk,rank_col,sum] + Scan parquet default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] InputAdapter Exchange [ss_item_sk] #2 WholeStageCodegen (1) - HashAggregate [ss_item_sk,ss_net_profit] [count,count,sum,sum] + HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] Project [ss_item_sk,ss_net_profit] Filter [ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_profit,ss_store_sk] + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) Project [item_sk,rnk] - Filter [item_sk,rnk] + Filter [rnk,item_sk] InputAdapter Window [rank_col] WholeStageCodegen (6) @@ -54,9 +54,9 @@ TakeOrderedAndProject [best_performing,rnk,worst_performing] Project [item_sk,rank_col] Filter [avg(ss_net_profit)] ReusedSubquery [rank_col] #1 - HashAggregate [count,ss_item_sk,sum] [avg(UnscaledValue(ss_net_profit)),avg(ss_net_profit),count,item_sk,rank_col,sum] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] InputAdapter - ReusedExchange [count,ss_item_sk,sum] #2 + ReusedExchange [ss_item_sk,sum,count] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 96e07ea3d8a33..f2106ad7aadd3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,13 +1,13 @@ -TakeOrderedAndProject [best_performing,rnk,worst_performing] +TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (10) - Project [i_product_name,i_product_name,rnk] - BroadcastHashJoin [i_item_sk,item_sk] - Project [i_product_name,item_sk,rnk] - BroadcastHashJoin [i_item_sk,item_sk] - Project [item_sk,item_sk,rnk] + Project [rnk,i_product_name,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [rnk,item_sk,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [item_sk,rnk,item_sk] BroadcastHashJoin [rnk,rnk] Project [item_sk,rnk] - Filter [item_sk,rnk] + Filter [rnk,item_sk] InputAdapter Window [rank_col] WholeStageCodegen (3) @@ -19,31 +19,31 @@ TakeOrderedAndProject [best_performing,rnk,worst_performing] Filter [avg(ss_net_profit)] Subquery #1 WholeStageCodegen (2) - HashAggregate [count,ss_store_sk,sum] [avg(UnscaledValue(ss_net_profit)),count,rank_col,sum] + HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] InputAdapter Exchange [ss_store_sk] #3 WholeStageCodegen (1) - HashAggregate [ss_net_profit,ss_store_sk] [count,count,sum,sum] - Project [ss_net_profit,ss_store_sk] - Filter [ss_addr_sk,ss_store_sk] + HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] + Project [ss_store_sk,ss_net_profit] + Filter [ss_store_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_net_profit,ss_store_sk] - HashAggregate [count,ss_item_sk,sum] [avg(UnscaledValue(ss_net_profit)),avg(ss_net_profit),count,item_sk,rank_col,sum] + Scan parquet default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] InputAdapter Exchange [ss_item_sk] #2 WholeStageCodegen (1) - HashAggregate [ss_item_sk,ss_net_profit] [count,count,sum,sum] + HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] Project [ss_item_sk,ss_net_profit] Filter [ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_profit,ss_store_sk] + Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) Project [item_sk,rnk] - Filter [item_sk,rnk] + Filter [rnk,item_sk] InputAdapter Window [rank_col] WholeStageCodegen (6) @@ -54,9 +54,9 @@ TakeOrderedAndProject [best_performing,rnk,worst_performing] Project [item_sk,rank_col] Filter [avg(ss_net_profit)] ReusedSubquery [rank_col] #1 - HashAggregate [count,ss_item_sk,sum] [avg(UnscaledValue(ss_net_profit)),avg(ss_net_profit),count,item_sk,rank_col,sum] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,avg(ss_net_profit),sum,count] InputAdapter - ReusedExchange [count,ss_item_sk,sum] #2 + ReusedExchange [ss_item_sk,sum,count] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt index 675ed20f170db..1eab468e67bc0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/simplified.txt @@ -1,52 +1,52 @@ -TakeOrderedAndProject [ca_city,ca_zip,sum(ws_sales_price)] +TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] WholeStageCodegen (13) - HashAggregate [ca_city,ca_zip,sum] [sum,sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price)] + HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] InputAdapter - Exchange [ca_city,ca_zip] #1 + Exchange [ca_zip,ca_city] #1 WholeStageCodegen (12) - HashAggregate [ca_city,ca_zip,ws_sales_price] [sum,sum] - Project [ca_city,ca_zip,ws_sales_price] + HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] + Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - Project [ca_city,ca_zip,i_item_id,ws_sales_price] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Project [ws_sales_price,ca_city,ca_zip,i_item_id] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (4) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #2 WholeStageCodegen (3) - Project [i_item_id,ws_bill_customer_sk,ws_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_bill_customer_sk,ws_item_sk,ws_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter WholeStageCodegen (10) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (9) - Project [c_customer_sk,ca_city,ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] + Project [ca_city,ca_zip,c_customer_sk] + SortMergeJoin [ca_address_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (6) Sort [ca_address_sk] @@ -63,10 +63,10 @@ TakeOrderedAndProject [ca_city,ca_zip,sum(ws_sales_price)] InputAdapter Exchange [c_current_addr_sk] #7 WholeStageCodegen (7) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #8 WholeStageCodegen (11) @@ -74,4 +74,4 @@ TakeOrderedAndProject [ca_city,ca_zip,sum(ws_sales_price)] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index 9b005f76afcdb..7cc474fb54bf9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [ca_city,ca_zip,sum(ws_sales_price)] +TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] WholeStageCodegen (7) - HashAggregate [ca_city,ca_zip,sum] [sum,sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price)] + HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] InputAdapter - Exchange [ca_city,ca_zip] #1 + Exchange [ca_zip,ca_city] #1 WholeStageCodegen (6) - HashAggregate [ca_city,ca_zip,ws_sales_price] [sum,sum] - Project [ca_city,ca_zip,ws_sales_price] + HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] + Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - Project [ca_city,ca_zip,i_item_id,ws_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ca_city,ca_zip,ws_item_sk,ws_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ca_city,ca_zip,ws_item_sk,ws_sales_price,ws_sold_date_sk] + Project [ws_sales_price,ca_city,ca_zip,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_sales_price,ca_city,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ws_item_sk,ws_sales_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_sales_price,c_current_addr_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Filter [ws_bill_customer_sk,ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -38,17 +38,17 @@ TakeOrderedAndProject [ca_city,ca_zip,sum(ws_sales_price)] BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) @@ -56,4 +56,4 @@ TakeOrderedAndProject [ca_city,ca_zip,sum(ws_sales_price)] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt index db6b41286a64e..5eef8b789842c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/simplified.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] +TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] WholeStageCodegen (16) - Project [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] - SortMergeJoin [bought_city,c_customer_sk,ca_city,ss_customer_sk] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + SortMergeJoin [c_customer_sk,ss_customer_sk,ca_city,bought_city] InputAdapter WholeStageCodegen (6) Sort [c_customer_sk] @@ -16,10 +16,10 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s InputAdapter Exchange [c_current_addr_sk] #2 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (4) Sort [ca_address_sk] @@ -36,34 +36,34 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (14) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,bought_city,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - HashAggregate [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - SortMergeJoin [ca_address_sk,ss_addr_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (11) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #5 WholeStageCodegen (10) - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #6 WholeStageCodegen (7) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -71,12 +71,12 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #8 WholeStageCodegen (9) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index edf9894f202ea..abdc7a3babccd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] +TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] WholeStageCodegen (8) - Project [amt,bought_city,c_first_name,c_last_name,ca_city,profit,ss_ticket_number] - BroadcastHashJoin [bought_city,c_current_addr_sk,ca_address_sk,ca_city] - Project [amt,bought_city,c_current_addr_sk,c_first_name,c_last_name,profit,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,bought_city,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] + Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] InputAdapter - Exchange [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #1 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 WholeStageCodegen (5) - HashAggregate [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -36,12 +36,12 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -55,9 +55,9 @@ TakeOrderedAndProject [amt,bought_city,c_first_name,c_last_name,ca_city,profit,s InputAdapter BroadcastExchange #6 WholeStageCodegen (6) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index 70261f5d29bd5..987dc71641b07 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -104,7 +104,7 @@ Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_sales_price#4, d_ Output [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_company_name), IsNotNull(s_store_name)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -112,7 +112,7 @@ Input [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] (12) Filter [codegen id : 2] Input [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] -Condition : ((isnotnull(s_store_sk#9) AND isnotnull(s_company_name#11)) AND isnotnull(s_store_name#10)) +Condition : ((isnotnull(s_store_sk#9) AND isnotnull(s_store_name#10)) AND isnotnull(s_company_name#11)) (13) BroadcastExchange Input [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt index 8f2656dc29d18..e7ff97ff2a7c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/simplified.txt @@ -1,107 +1,107 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,psum,s_company_name,s_store_name,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (37) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (25) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,rn,s_company_name,s_store_name,sum_sales,sum_sales] - SortMergeJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (13) - Sort [i_brand,i_category,rn,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_brand,i_category,rn,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #1 WholeStageCodegen (12) - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (11) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #2 + Exchange [i_category,i_brand,s_store_name,s_company_name] #2 WholeStageCodegen (10) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] WholeStageCodegen (9) - Sort [d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year] InputAdapter - Exchange [d_year,i_brand,i_category,s_company_name,s_store_name] #3 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year] #3 WholeStageCodegen (8) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] #4 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #4 WholeStageCodegen (7) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 WholeStageCodegen (3) - Project [d_moy,d_year,s_company_name,s_store_name,ss_item_sk,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_year,ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) - Filter [s_company_name,s_store_name,s_store_sk] + Filter [s_store_sk,s_store_name,s_company_name] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #8 WholeStageCodegen (5) - Filter [i_brand,i_category,i_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (24) - Sort [i_brand,i_category,rn,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_brand,i_category,rn,s_company_name,s_store_name] #9 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #9 WholeStageCodegen (23) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (22) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name] #10 WholeStageCodegen (21) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] #4 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #4 InputAdapter WholeStageCodegen (36) - Sort [i_brand,i_category,rn,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_brand,i_category,rn,s_company_name,s_store_name] #11 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #11 WholeStageCodegen (35) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (34) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] #10 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 478e717998aac..a84ba3a03a674 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -124,7 +124,7 @@ Input [8]: [i_brand#2, i_category#3, ss_sold_date_sk#4, ss_store_sk#6, ss_sales_ Output [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_company_name), IsNotNull(s_store_name)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] (18) Filter [codegen id : 3] Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] -Condition : ((isnotnull(s_store_sk#13) AND isnotnull(s_company_name#15)) AND isnotnull(s_store_name#14)) +Condition : ((isnotnull(s_store_sk#13) AND isnotnull(s_store_name#14)) AND isnotnull(s_company_name#15)) (19) BroadcastExchange Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] @@ -195,7 +195,7 @@ Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24, rn#26] -Condition : (((((isnotnull(avg_monthly_sales#24) AND isnotnull(d_year#10)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#24 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#26)) +Condition : (((((isnotnull(d_year#10) AND isnotnull(avg_monthly_sales#24)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#24 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#26)) (33) ReusedExchange [Reuses operator id: 23] Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index c2cb621d55fcd..66ccfa4a9f353 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,84 +1,84 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,psum,s_company_name,s_store_name,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] WholeStageCodegen (23) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,rn,s_company_name,s_store_name,sum_sales,sum_sales] - BroadcastHashJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (8) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 WholeStageCodegen (7) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] WholeStageCodegen (6) - Sort [d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year] InputAdapter - Exchange [d_year,i_brand,i_category,s_company_name,s_store_name] #2 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year] #2 WholeStageCodegen (5) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] #3 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3 WholeStageCodegen (4) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_year,i_brand,i_category,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_category,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_brand,i_category,i_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - Filter [s_company_name,s_store_name,s_store_sk] + Filter [s_store_sk,s_store_name,s_company_name] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #7 WholeStageCodegen (15) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (14) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name] #8 WholeStageCodegen (13) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] #3 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter BroadcastExchange #9 WholeStageCodegen (22) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (21) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] #8 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/simplified.txt index 0a68c861c2844..710fbdd72e07b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/simplified.txt @@ -1,21 +1,21 @@ WholeStageCodegen (6) - HashAggregate [sum] [sum,sum(cast(ss_quantity as bigint)),sum(ss_quantity)] + HashAggregate [sum] [sum(cast(ss_quantity as bigint)),sum(ss_quantity),sum] InputAdapter Exchange #1 WholeStageCodegen (5) HashAggregate [ss_quantity] [sum,sum] Project [ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ca_address_sk,ca_state,ss_addr_sk,ss_net_profit] - Project [ss_addr_sk,ss_net_profit,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cd_education_status,cd_marital_status,ss_cdemo_sk,ss_sales_price] - Project [ss_addr_sk,ss_cdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_addr_sk,ss_cdemo_sk,ss_net_profit,ss_sales_price,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_sold_date_sk,ss_addr_sk,ss_quantity,ss_net_profit] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_sold_date_sk,ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sold_date_sk,ss_sales_price,ss_net_profit] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -26,23 +26,23 @@ WholeStageCodegen (6) InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [ca_address_sk,ca_state] - Filter [ca_address_sk,ca_country,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 0a68c861c2844..710fbdd72e07b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -1,21 +1,21 @@ WholeStageCodegen (6) - HashAggregate [sum] [sum,sum(cast(ss_quantity as bigint)),sum(ss_quantity)] + HashAggregate [sum] [sum(cast(ss_quantity as bigint)),sum(ss_quantity),sum] InputAdapter Exchange #1 WholeStageCodegen (5) HashAggregate [ss_quantity] [sum,sum] Project [ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ca_address_sk,ca_state,ss_addr_sk,ss_net_profit] - Project [ss_addr_sk,ss_net_profit,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cd_education_status,cd_marital_status,ss_cdemo_sk,ss_sales_price] - Project [ss_addr_sk,ss_cdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_addr_sk,ss_cdemo_sk,ss_net_profit,ss_sales_price,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_sold_date_sk,ss_addr_sk,ss_quantity,ss_net_profit] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_sold_date_sk,ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sold_date_sk,ss_sales_price,ss_net_profit] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_net_profit,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -26,23 +26,23 @@ WholeStageCodegen (6) InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [ca_address_sk,ca_state] - Filter [ca_address_sk,ca_country,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index 2303cae3725e5..114b5e0568223 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -92,7 +92,7 @@ TakeOrderedAndProject (87) Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_quantity), IsNotNull(ws_net_paid), IsNotNull(ws_net_profit), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] @@ -100,7 +100,7 @@ Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, w (3) Filter [codegen id : 2] Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] -Condition : ((((((((isnotnull(ws_quantity#4) AND isnotnull(ws_net_paid#5)) AND isnotnull(ws_net_profit#6)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_sold_date_sk#1)) +Condition : ((((((((isnotnull(ws_net_profit#6) AND isnotnull(ws_net_paid#5)) AND isnotnull(ws_quantity#4)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_sold_date_sk#1)) (4) Project [codegen id : 2] Output [5]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5] @@ -149,7 +149,7 @@ Arguments: [cast(ws_order_number#3 as bigint) ASC NULLS FIRST, cast(ws_item_sk#2 Output [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_amt#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -157,7 +157,7 @@ Input [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_ (16) Filter [codegen id : 4] Input [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_amt#15] -Condition : (((isnotnull(wr_return_amt#15) AND (wr_return_amt#15 > 10000.00)) AND isnotnull(wr_item_sk#12)) AND isnotnull(wr_order_number#13)) +Condition : (((isnotnull(wr_return_amt#15) AND (wr_return_amt#15 > 10000.00)) AND isnotnull(wr_order_number#13)) AND isnotnull(wr_item_sk#12)) (17) Exchange Input [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_amt#15] @@ -226,7 +226,7 @@ Input [5]: [item#34, return_ratio#35, currency_ratio#36, return_rank#38, currenc Output [6]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_net_profit#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_item_sk), IsNotNull(cs_order_number), IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 12] @@ -234,7 +234,7 @@ Input [6]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#4 (33) Filter [codegen id : 12] Input [6]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_net_profit#46] -Condition : ((((((((isnotnull(cs_net_profit#46) AND isnotnull(cs_net_paid#45)) AND isnotnull(cs_quantity#44)) AND (cs_net_profit#46 > 1.00)) AND (cs_net_paid#45 > 0.00)) AND (cs_quantity#44 > 0)) AND isnotnull(cs_item_sk#42)) AND isnotnull(cs_order_number#43)) AND isnotnull(cs_sold_date_sk#41)) +Condition : ((((((((isnotnull(cs_net_profit#46) AND isnotnull(cs_net_paid#45)) AND isnotnull(cs_quantity#44)) AND (cs_net_profit#46 > 1.00)) AND (cs_net_paid#45 > 0.00)) AND (cs_quantity#44 > 0)) AND isnotnull(cs_order_number#43)) AND isnotnull(cs_item_sk#42)) AND isnotnull(cs_sold_date_sk#41)) (34) Project [codegen id : 12] Output [5]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45] @@ -264,7 +264,7 @@ Arguments: [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST], Output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_quantity#50, cr_return_amount#51] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 14] @@ -272,7 +272,7 @@ Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_quantity#50, cr_return_ (42) Filter [codegen id : 14] Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_quantity#50, cr_return_amount#51] -Condition : (((isnotnull(cr_return_amount#51) AND (cr_return_amount#51 > 10000.00)) AND isnotnull(cr_item_sk#48)) AND isnotnull(cr_order_number#49)) +Condition : (((isnotnull(cr_return_amount#51) AND (cr_return_amount#51 > 10000.00)) AND isnotnull(cr_order_number#49)) AND isnotnull(cr_item_sk#48)) (43) Exchange Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_quantity#50, cr_return_amount#51] @@ -341,7 +341,7 @@ Input [5]: [item#70, return_ratio#71, currency_ratio#72, return_rank#74, currenc Output [6]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 22] @@ -349,7 +349,7 @@ Input [6]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity# (59) Filter [codegen id : 22] Input [6]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82] -Condition : ((((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_item_sk#78)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_sold_date_sk#77)) +Condition : ((((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) AND isnotnull(ss_sold_date_sk#77)) (60) Project [codegen id : 22] Output [5]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt index 7fc20fb4df8d2..ab300ca150457 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (32) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter - Exchange [channel,currency_rank,item,return_rank,return_ratio] #1 + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 WholeStageCodegen (31) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union WholeStageCodegen (10) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (9) @@ -21,47 +21,47 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #2 WholeStageCodegen (7) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,sum,ws_item_sk] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (6) - HashAggregate [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter WholeStageCodegen (3) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #4 + Exchange [ws_order_number,ws_item_sk] #4 WholeStageCodegen (2) - Project [ws_item_sk,ws_net_paid,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_item_sk,ws_net_paid,ws_order_number,ws_quantity,ws_sold_date_sk] - Filter [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter WholeStageCodegen (5) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_item_sk,wr_order_number] #6 + Exchange [wr_order_number,wr_item_sk] #6 WholeStageCodegen (4) - Filter [wr_item_sk,wr_order_number,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] WholeStageCodegen (20) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (19) @@ -73,41 +73,41 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #7 WholeStageCodegen (17) - HashAggregate [cs_item_sk,isEmpty,isEmpty,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #8 WholeStageCodegen (16) - HashAggregate [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter WholeStageCodegen (13) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #9 + Exchange [cs_order_number,cs_item_sk] #9 WholeStageCodegen (12) - Project [cs_item_sk,cs_net_paid,cs_order_number,cs_quantity] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_net_paid,cs_order_number,cs_quantity,cs_sold_date_sk] - Filter [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter WholeStageCodegen (15) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #10 + Exchange [cr_order_number,cr_item_sk] #10 WholeStageCodegen (14) - Filter [cr_item_sk,cr_order_number,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] WholeStageCodegen (30) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (29) @@ -119,35 +119,35 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #11 WholeStageCodegen (27) - HashAggregate [isEmpty,isEmpty,ss_item_sk,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ss_item_sk] #12 WholeStageCodegen (26) - HashAggregate [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (23) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #13 + Exchange [ss_ticket_number,ss_item_sk] #13 WholeStageCodegen (22) - Project [ss_item_sk,ss_net_paid,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_item_sk,ss_net_paid,ss_quantity,ss_sold_date_sk,ss_ticket_number] - Filter [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter WholeStageCodegen (25) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #14 + Exchange [sr_ticket_number,sr_item_sk] #14 WholeStageCodegen (24) - Filter [sr_item_sk,sr_return_amt,sr_ticket_number] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 909041abb5862..8d10c1641ef46 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -83,7 +83,7 @@ TakeOrderedAndProject (78) Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_net_paid), IsNotNull(ws_quantity), IsNotNull(ws_net_profit), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 3] @@ -91,7 +91,7 @@ Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, w (3) Filter [codegen id : 3] Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] -Condition : ((((((((isnotnull(ws_net_paid#5) AND isnotnull(ws_quantity#4)) AND isnotnull(ws_net_profit#6)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_sold_date_sk#1)) +Condition : ((((((((isnotnull(ws_net_profit#6) AND isnotnull(ws_net_paid#5)) AND isnotnull(ws_quantity#4)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_sold_date_sk#1)) (4) Project [codegen id : 3] Output [5]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5] @@ -205,7 +205,7 @@ Input [5]: [item#33, return_ratio#34, currency_ratio#35, return_rank#37, currenc Output [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_net_paid), IsNotNull(cs_net_profit), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_item_sk), IsNotNull(cs_order_number), IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 10] @@ -213,7 +213,7 @@ Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#4 (30) Filter [codegen id : 10] Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] -Condition : ((((((((isnotnull(cs_quantity#43) AND isnotnull(cs_net_paid#44)) AND isnotnull(cs_net_profit#45)) AND (cs_net_profit#45 > 1.00)) AND (cs_net_paid#44 > 0.00)) AND (cs_quantity#43 > 0)) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_order_number#42)) AND isnotnull(cs_sold_date_sk#40)) +Condition : ((((((((isnotnull(cs_net_profit#45) AND isnotnull(cs_net_paid#44)) AND isnotnull(cs_quantity#43)) AND (cs_net_profit#45 > 1.00)) AND (cs_net_paid#44 > 0.00)) AND (cs_quantity#43 > 0)) AND isnotnull(cs_order_number#42)) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_sold_date_sk#40)) (31) Project [codegen id : 10] Output [5]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44] @@ -223,7 +223,7 @@ Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#4 Output [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 8] @@ -231,7 +231,7 @@ Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_ (34) Filter [codegen id : 8] Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] -Condition : (((isnotnull(cr_return_amount#49) AND (cr_return_amount#49 > 10000.00)) AND isnotnull(cr_item_sk#46)) AND isnotnull(cr_order_number#47)) +Condition : (((isnotnull(cr_return_amount#49) AND (cr_return_amount#49 > 10000.00)) AND isnotnull(cr_order_number#47)) AND isnotnull(cr_item_sk#46)) (35) BroadcastExchange Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] @@ -308,7 +308,7 @@ Input [5]: [item#68, return_ratio#69, currency_ratio#70, return_rank#72, currenc Output [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_net_paid), IsNotNull(ss_quantity), IsNotNull(ss_net_profit), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 17] @@ -316,7 +316,7 @@ Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity# (53) Filter [codegen id : 17] Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] -Condition : ((((((((isnotnull(ss_net_paid#79) AND isnotnull(ss_quantity#78)) AND isnotnull(ss_net_profit#80)) AND (ss_net_profit#80 > 1.00)) AND (ss_net_paid#79 > 0.00)) AND (ss_quantity#78 > 0)) AND isnotnull(ss_item_sk#76)) AND isnotnull(ss_ticket_number#77)) AND isnotnull(ss_sold_date_sk#75)) +Condition : ((((((((isnotnull(ss_net_profit#80) AND isnotnull(ss_net_paid#79)) AND isnotnull(ss_quantity#78)) AND (ss_net_profit#80 > 1.00)) AND (ss_net_paid#79 > 0.00)) AND (ss_quantity#78 > 0)) AND isnotnull(ss_ticket_number#77)) AND isnotnull(ss_item_sk#76)) AND isnotnull(ss_sold_date_sk#75)) (54) Project [codegen id : 17] Output [5]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index acba83ae8e411..c15f2394e1a44 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (23) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter - Exchange [channel,currency_rank,item,return_rank,return_ratio] #1 + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 WholeStageCodegen (22) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union WholeStageCodegen (7) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (6) @@ -21,38 +21,38 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #2 WholeStageCodegen (4) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,sum,ws_item_sk] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (3) - HashAggregate [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Project [ws_item_sk,ws_net_paid,ws_order_number,ws_quantity,ws_sold_date_sk] - Filter [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [wr_item_sk,wr_order_number,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (14) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (13) @@ -64,32 +64,32 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #6 WholeStageCodegen (11) - HashAggregate [cs_item_sk,isEmpty,isEmpty,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #7 WholeStageCodegen (10) - HashAggregate [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_item_sk,cs_net_paid,cs_order_number,cs_quantity,cs_sold_date_sk] - Filter [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) - Filter [cr_item_sk,cr_order_number,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (21) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (20) @@ -101,26 +101,26 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #9 WholeStageCodegen (18) - HashAggregate [isEmpty,isEmpty,ss_item_sk,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ss_item_sk] #10 WholeStageCodegen (17) - HashAggregate [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Project [ss_item_sk,ss_net_paid,ss_quantity,ss_sold_date_sk,ss_ticket_number] - Filter [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit] InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - Filter [sr_item_sk,sr_return_amt,sr_ticket_number] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] InputAdapter ReusedExchange [d_date_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index 71e204433c6ef..80b07a3712d36 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -1,37 +1,37 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (24) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,spark_grouping_id,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 WholeStageCodegen (23) - HashAggregate [channel,id,profit,returns,sales,spark_grouping_id] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Expand [channel,id,profit,returns,sales] + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] InputAdapter Union WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [RETURNS,channel,id,profit,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + HashAggregate [s_store_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] InputAdapter Exchange [s_store_id] #2 WholeStageCodegen (5) - HashAggregate [net_loss,profit,return_amt,s_store_id,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,s_store_id,sales_price] - BroadcastHashJoin [s_store_sk,store_sk] - Project [net_loss,profit,return_amt,sales_price,store_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (1) - Project [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] WholeStageCodegen (2) - Project [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) @@ -39,38 +39,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [RETURNS,channel,id,profit,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [cp_catalog_page_id] #5 WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] - BroadcastHashJoin [cp_catalog_page_sk,page_sk] - Project [net_loss,page_sk,profit,return_amt,sales_price] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - Filter [cs_catalog_page_sk,cs_sold_date_sk] + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_sold_date_sk,cs_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] - Filter [cr_catalog_page_sk,cr_returned_date_sk] + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_returned_date_sk,cr_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -79,27 +79,27 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (22) - HashAggregate [sum,sum,sum,sum,web_site_id] [RETURNS,channel,id,profit,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [web_site_id] #7 WholeStageCodegen (21) - HashAggregate [net_loss,profit,return_amt,sales_price,web_site_id] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,sales_price,web_site_id] - BroadcastHashJoin [web_site_sk,wsr_web_site_sk] - Project [net_loss,profit,return_amt,sales_price,wsr_web_site_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (13) - Project [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_sold_date_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit] WholeStageCodegen (18) - Project [wr_net_loss,wr_return_amt,wr_returned_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter WholeStageCodegen (15) @@ -110,7 +110,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [wr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] @@ -120,7 +120,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_order_number,ws_web_site_sk] + Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -129,4 +129,4 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 0bb6f48db3e12..9b7cc3360367c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -1,37 +1,37 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,spark_grouping_id,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 WholeStageCodegen (20) - HashAggregate [channel,id,profit,returns,sales,spark_grouping_id] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Expand [channel,id,profit,returns,sales] + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] InputAdapter Union WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [RETURNS,channel,id,profit,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + HashAggregate [s_store_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] InputAdapter Exchange [s_store_id] #2 WholeStageCodegen (5) - HashAggregate [net_loss,profit,return_amt,s_store_id,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,s_store_id,sales_price] - BroadcastHashJoin [s_store_sk,store_sk] - Project [net_loss,profit,return_amt,sales_price,store_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (1) - Project [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] WholeStageCodegen (2) - Project [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) @@ -39,38 +39,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [RETURNS,channel,id,profit,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [cp_catalog_page_id] #5 WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] - BroadcastHashJoin [cp_catalog_page_sk,page_sk] - Project [net_loss,page_sk,profit,return_amt,sales_price] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - Filter [cs_catalog_page_sk,cs_sold_date_sk] + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_sold_date_sk,cs_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] - Filter [cr_catalog_page_sk,cr_returned_date_sk] + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_returned_date_sk,cr_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -79,39 +79,39 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (19) - HashAggregate [sum,sum,sum,sum,web_site_id] [RETURNS,channel,id,profit,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [web_site_id] #7 WholeStageCodegen (18) - HashAggregate [net_loss,profit,return_amt,sales_price,web_site_id] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,sales_price,web_site_id] - BroadcastHashJoin [web_site_sk,wsr_web_site_sk] - Project [net_loss,profit,return_amt,sales_price,wsr_web_site_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (13) - Project [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_sold_date_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit] WholeStageCodegen (15) - Project [wr_net_loss,wr_return_amt,wr_returned_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] Filter [wr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter BroadcastExchange #8 WholeStageCodegen (14) Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_order_number,ws_web_site_sk] + Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -120,4 +120,4 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt index 5438ebb4c0910..df1197d7c925e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt @@ -40,7 +40,7 @@ TakeOrderedAndProject (35) Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 3] @@ -48,7 +48,7 @@ Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss (3) Filter [codegen id : 3] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] -Condition : ((((isnotnull(ss_customer_sk#3) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_sold_date_sk#1)) +Condition : ((((isnotnull(ss_ticket_number#5) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_sold_date_sk#1)) (4) Scan parquet default.date_dim Output [1]: [d_date_sk#6] @@ -138,7 +138,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Output [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (24) ColumnarToRow @@ -146,7 +146,7 @@ Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_ (25) Filter Input [4]: [sr_returned_date_sk#25, sr_item_sk#26, sr_customer_sk#27, sr_ticket_number#28] -Condition : (((isnotnull(sr_ticket_number#28) AND isnotnull(sr_customer_sk#27)) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_returned_date_sk#25)) +Condition : (((isnotnull(sr_ticket_number#28) AND isnotnull(sr_item_sk#26)) AND isnotnull(sr_customer_sk#27)) AND isnotnull(sr_returned_date_sk#25)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cast(d_date_sk#21 as bigint)] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt index 460ed30f87df5..5d6f38e882a5c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/simplified.txt @@ -1,26 +1,26 @@ -TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip] +TakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (9) - HashAggregate [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sum,sum,sum,sum,sum] [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum,sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 30) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 60) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 90) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] + HashAggregate [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] [sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 30) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 60) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 90) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 120) THEN 1 ELSE 0 END as bigint)),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter - Exchange [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip] #1 + Exchange [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] #1 WholeStageCodegen (8) - HashAggregate [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sr_returned_date_sk,ss_sold_date_sk] - SortMergeJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] + HashAggregate [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] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [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] + SortMergeJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] InputAdapter WholeStageCodegen (4) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk,ss_customer_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_item_sk,ss_customer_sk] #2 WholeStageCodegen (3) - Project [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ticket_number,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] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -34,24 +34,24 @@ TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_company_id,s_county,s_state,s_store_name,s_store_sk,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip] + Scan parquet 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] InputAdapter WholeStageCodegen (7) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk,sr_customer_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #5 + Exchange [sr_ticket_number,sr_item_sk,sr_customer_sk] #5 WholeStageCodegen (6) - Project [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Project [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] BroadcastHashJoin [d_date_sk,sr_returned_date_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index eac796a4eff48..e083affa7261d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 5] @@ -45,13 +45,13 @@ Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss (3) Filter [codegen id : 5] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5] -Condition : ((((isnotnull(ss_item_sk#2) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_ticket_number#5)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_sold_date_sk#1)) +Condition : ((((isnotnull(ss_ticket_number#5) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) AND isnotnull(ss_sold_date_sk#1)) (4) Scan parquet default.store_returns Output [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_returned_date_sk)] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk), IsNotNull(sr_returned_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -59,7 +59,7 @@ Input [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_num (6) Filter [codegen id : 1] Input [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] -Condition : (((isnotnull(sr_ticket_number#9) AND isnotnull(sr_customer_sk#8)) AND isnotnull(sr_item_sk#7)) AND isnotnull(sr_returned_date_sk#6)) +Condition : (((isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#7)) AND isnotnull(sr_customer_sk#8)) AND isnotnull(sr_returned_date_sk#6)) (7) BroadcastExchange Input [4]: [sr_returned_date_sk#6, sr_item_sk#7, sr_customer_sk#8, sr_ticket_number#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index 1ef21d7b18712..43e7773855595 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip] +TakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) - HashAggregate [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sum,sum,sum,sum,sum] [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum,sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 30) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 60) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 90) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 120) THEN 1 ELSE 0 END as bigint))] + HashAggregate [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] [sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 30) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 60) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 90) AND ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((sr_returned_date_skL - cast(ss_sold_date_sk as bigint)) > 120) THEN 1 ELSE 0 END as bigint)),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter - Exchange [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip] #1 + Exchange [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] #1 WholeStageCodegen (5) - HashAggregate [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sr_returned_date_sk,ss_sold_date_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sr_returned_date_sk,ss_sold_date_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [s_city,s_company_id,s_county,s_state,s_store_name,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip,sr_returned_date_sk,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [sr_returned_date_sk,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,sr_ticket_number,ss_customer_sk,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + HashAggregate [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] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [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] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [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] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_sold_date_sk,ss_store_sk,sr_returned_date_sk] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] + Filter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk,sr_customer_sk,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_customer_sk,sr_item_sk,sr_returned_date_sk,sr_ticket_number] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_company_id,s_county,s_state,s_store_name,s_store_sk,s_street_name,s_street_number,s_street_type,s_suite_number,s_zip] + Scan parquet 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] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -42,7 +42,7 @@ TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/simplified.txt index 42138ddee3e93..17f35514f4599 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/simplified.txt @@ -1,71 +1,71 @@ -TakeOrderedAndProject [d_date,item_sk,store_cumulative,store_sales,web_cumulative,web_sales] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] WholeStageCodegen (15) - Filter [store_cumulative,web_cumulative] + Filter [web_cumulative,store_cumulative] InputAdapter - Window [d_date,item_sk,store_sales,web_sales] + Window [web_sales,item_sk,d_date,store_sales] WholeStageCodegen (14) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 WholeStageCodegen (13) - Project [cume_sales,cume_sales,d_date,d_date,item_sk,item_sk] + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] InputAdapter - SortMergeJoin [d_date,d_date,item_sk,item_sk] + SortMergeJoin [item_sk,d_date,item_sk,d_date] WholeStageCodegen (6) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #2 + Exchange [item_sk,d_date] #2 WholeStageCodegen (5) - Project [cume_sales,d_date,item_sk] + Project [item_sk,d_date,cume_sales] InputAdapter - Window [_w0,d_date,ws_item_sk] + Window [_w0,ws_item_sk,d_date] WholeStageCodegen (4) - Sort [d_date,ws_item_sk] + Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (3) - HashAggregate [d_date,sum,ws_item_sk] [_w0,item_sk,sum,sum(UnscaledValue(ws_sales_price))] + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] InputAdapter - Exchange [d_date,ws_item_sk] #4 + Exchange [ws_item_sk,d_date] #4 WholeStageCodegen (2) - HashAggregate [d_date,ws_item_sk,ws_sales_price] [sum,sum] - Project [d_date,ws_item_sk,ws_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] WholeStageCodegen (12) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #6 + Exchange [item_sk,d_date] #6 WholeStageCodegen (11) - Project [cume_sales,d_date,item_sk] + Project [item_sk,d_date,cume_sales] InputAdapter - Window [_w0,d_date,ss_item_sk] + Window [_w0,ss_item_sk,d_date] WholeStageCodegen (10) - Sort [d_date,ss_item_sk] + Sort [ss_item_sk,d_date] InputAdapter Exchange [ss_item_sk] #7 WholeStageCodegen (9) - HashAggregate [d_date,ss_item_sk,sum] [_w0,item_sk,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] InputAdapter - Exchange [d_date,ss_item_sk] #8 + Exchange [ss_item_sk,d_date] #8 WholeStageCodegen (8) - HashAggregate [d_date,ss_item_sk,ss_sales_price] [sum,sum] - Project [d_date,ss_item_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] InputAdapter - ReusedExchange [d_date,d_date_sk] #5 + ReusedExchange [d_date_sk,d_date] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index 42138ddee3e93..17f35514f4599 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,71 +1,71 @@ -TakeOrderedAndProject [d_date,item_sk,store_cumulative,store_sales,web_cumulative,web_sales] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] WholeStageCodegen (15) - Filter [store_cumulative,web_cumulative] + Filter [web_cumulative,store_cumulative] InputAdapter - Window [d_date,item_sk,store_sales,web_sales] + Window [web_sales,item_sk,d_date,store_sales] WholeStageCodegen (14) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 WholeStageCodegen (13) - Project [cume_sales,cume_sales,d_date,d_date,item_sk,item_sk] + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] InputAdapter - SortMergeJoin [d_date,d_date,item_sk,item_sk] + SortMergeJoin [item_sk,d_date,item_sk,d_date] WholeStageCodegen (6) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #2 + Exchange [item_sk,d_date] #2 WholeStageCodegen (5) - Project [cume_sales,d_date,item_sk] + Project [item_sk,d_date,cume_sales] InputAdapter - Window [_w0,d_date,ws_item_sk] + Window [_w0,ws_item_sk,d_date] WholeStageCodegen (4) - Sort [d_date,ws_item_sk] + Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (3) - HashAggregate [d_date,sum,ws_item_sk] [_w0,item_sk,sum,sum(UnscaledValue(ws_sales_price))] + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] InputAdapter - Exchange [d_date,ws_item_sk] #4 + Exchange [ws_item_sk,d_date] #4 WholeStageCodegen (2) - HashAggregate [d_date,ws_item_sk,ws_sales_price] [sum,sum] - Project [d_date,ws_item_sk,ws_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] WholeStageCodegen (12) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #6 + Exchange [item_sk,d_date] #6 WholeStageCodegen (11) - Project [cume_sales,d_date,item_sk] + Project [item_sk,d_date,cume_sales] InputAdapter - Window [_w0,d_date,ss_item_sk] + Window [_w0,ss_item_sk,d_date] WholeStageCodegen (10) - Sort [d_date,ss_item_sk] + Sort [ss_item_sk,d_date] InputAdapter Exchange [ss_item_sk] #7 WholeStageCodegen (9) - HashAggregate [d_date,ss_item_sk,sum] [_w0,item_sk,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] InputAdapter - Exchange [d_date,ss_item_sk] #8 + Exchange [ss_item_sk,d_date] #8 WholeStageCodegen (8) - HashAggregate [d_date,ss_item_sk,ss_sales_price] [sum,sum] - Project [d_date,ss_item_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] InputAdapter - ReusedExchange [d_date,d_date_sk] #5 + ReusedExchange [d_date_sk,d_date] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt index 0746a032aa1a2..8ed500d84390c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,ext_price] +TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_brand_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 7d70658c38241..ba48ad5b699f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,d_year,ext_price] +TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [d_year,i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [d_year,i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_year,ss_ext_sales_price,ss_item_sk] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk,d_year] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/simplified.txt index 98f126325517b..ca187424ad900 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (7) - Project [avg_quarterly_sales,i_manufact_id,sum_sales] + Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] @@ -9,29 +9,29 @@ TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] InputAdapter Exchange [i_manufact_id] #1 WholeStageCodegen (5) - HashAggregate [d_qoy,i_manufact_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_qoy,i_manufact_id] #2 + Exchange [i_manufact_id,d_qoy] #2 WholeStageCodegen (4) - HashAggregate [d_qoy,i_manufact_id,ss_sales_price] [sum,sum] - Project [d_qoy,i_manufact_id,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manufact_id,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_manufact_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_sold_date_sk,ss_sales_price] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [i_item_sk,i_manufact_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -43,7 +43,7 @@ TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk,d_qoy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_qoy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index c8070c46c9c80..cd55b60d2580f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] WholeStageCodegen (7) - Project [avg_quarterly_sales,i_manufact_id,sum_sales] + Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] @@ -9,34 +9,34 @@ TakeOrderedAndProject [avg_quarterly_sales,i_manufact_id,sum_sales] InputAdapter Exchange [i_manufact_id] #1 WholeStageCodegen (5) - HashAggregate [d_qoy,i_manufact_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_qoy,i_manufact_id] #2 + Exchange [i_manufact_id,d_qoy] #2 WholeStageCodegen (4) - HashAggregate [d_qoy,i_manufact_id,ss_sales_price] [sum,sum] - Project [d_qoy,i_manufact_id,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_qoy,i_manufact_id,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manufact_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] Project [i_item_sk,i_manufact_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_qoy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_qoy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index bdaf52e928d2e..d78565986bc0a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -272,7 +272,7 @@ Arguments: [c_current_addr_sk#23 ASC NULLS FIRST], false, 0 Output [3]: [ca_address_sk#30, ca_county#31, ca_state#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state), IsNotNull(ca_county)] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct (46) ColumnarToRow [codegen id : 14] @@ -280,7 +280,7 @@ Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] (47) Filter [codegen id : 14] Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] -Condition : ((isnotnull(ca_address_sk#30) AND isnotnull(ca_state#32)) AND isnotnull(ca_county#31)) +Condition : ((isnotnull(ca_address_sk#30) AND isnotnull(ca_county#31)) AND isnotnull(ca_state#32)) (48) Exchange Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] @@ -303,7 +303,7 @@ Input [7]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_s Output [2]: [s_county#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_county)] +PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct (53) ColumnarToRow [codegen id : 16] @@ -311,7 +311,7 @@ Input [2]: [s_county#34, s_state#35] (54) Filter [codegen id : 16] Input [2]: [s_county#34, s_state#35] -Condition : (isnotnull(s_state#35) AND isnotnull(s_county#34)) +Condition : (isnotnull(s_county#34) AND isnotnull(s_state#35)) (55) BroadcastExchange Input [2]: [s_county#34, s_state#35] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt index 1f0920d4cbbd6..cb7130f53c9a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt @@ -1,20 +1,20 @@ -TakeOrderedAndProject [num_customers,segment,segment_base] +TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (20) - HashAggregate [count,segment] [count,count(1),num_customers,segment_base] + HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 WholeStageCodegen (19) HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [segment,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] InputAdapter Exchange [c_customer_sk] #2 WholeStageCodegen (18) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price] BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ca_county,ca_state,ss_ext_sales_price,ss_sold_date_sk] + Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price,ca_county,ca_state] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (13) @@ -22,15 +22,15 @@ TakeOrderedAndProject [num_customers,segment,segment_base] InputAdapter Exchange [c_current_addr_sk] #3 WholeStageCodegen (12) - Project [c_current_addr_sk,c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + Project [c_customer_sk,c_current_addr_sk,ss_sold_date_sk,ss_ext_sales_price] SortMergeJoin [c_customer_sk,ss_customer_sk] InputAdapter WholeStageCodegen (9) Sort [c_customer_sk] - HashAggregate [c_current_addr_sk,c_customer_sk] - HashAggregate [c_current_addr_sk,c_customer_sk] - Project [c_current_addr_sk,c_customer_sk] - SortMergeJoin [c_customer_sk,customer_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + SortMergeJoin [customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [customer_sk] @@ -38,23 +38,23 @@ TakeOrderedAndProject [num_customers,segment,segment_base] Exchange [customer_sk] #4 WholeStageCodegen (5) Project [customer_sk] - BroadcastHashJoin [d_date_sk,sold_date_sk] - Project [customer_sk,sold_date_sk] - BroadcastHashJoin [i_item_sk,item_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] InputAdapter Union WholeStageCodegen (1) - Project [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] WholeStageCodegen (2) - Project [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -62,25 +62,25 @@ TakeOrderedAndProject [num_customers,segment,segment_base] Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #7 WholeStageCodegen (7) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] @@ -90,7 +90,7 @@ TakeOrderedAndProject [num_customers,segment,segment_base] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_sales_price] InputAdapter WholeStageCodegen (15) Sort [ca_address_sk] @@ -112,7 +112,7 @@ TakeOrderedAndProject [num_customers,segment,segment_base] BroadcastExchange #11 WholeStageCodegen (17) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] Subquery #1 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -121,10 +121,10 @@ TakeOrderedAndProject [num_customers,segment,segment_base] WholeStageCodegen (1) HashAggregate [(d_month_seq + 1)] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] Subquery #2 WholeStageCodegen (2) HashAggregate [(d_month_seq + 3)] @@ -133,10 +133,10 @@ TakeOrderedAndProject [num_customers,segment,segment_base] WholeStageCodegen (1) HashAggregate [(d_month_seq + 3)] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index b7c9373a8b4ec..2e93e26056615 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -268,7 +268,7 @@ Input [7]: [c_customer_sk#21, c_current_addr_sk#22, ss_sold_date_sk#25, ss_ext_s Output [2]: [s_county#33, s_state#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_county)] +PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct (46) ColumnarToRow [codegen id : 9] @@ -276,7 +276,7 @@ Input [2]: [s_county#33, s_state#34] (47) Filter [codegen id : 9] Input [2]: [s_county#33, s_state#34] -Condition : (isnotnull(s_state#34) AND isnotnull(s_county#33)) +Condition : (isnotnull(s_county#33) AND isnotnull(s_state#34)) (48) BroadcastExchange Input [2]: [s_county#33, s_state#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index fafabe1ef0515..dd92c52db4c7e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [num_customers,segment,segment_base] +TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (13) - HashAggregate [count,segment] [count,count(1),num_customers,segment_base] + HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 WholeStageCodegen (12) HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [segment,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] InputAdapter Exchange [c_customer_sk] #2 WholeStageCodegen (11) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price] BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ca_county,ca_state,ss_ext_sales_price,ss_sold_date_sk] + Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price,ca_county,ca_state] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + Project [c_customer_sk,c_current_addr_sk,ss_sold_date_sk,ss_ext_sales_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [c_current_addr_sk,c_customer_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] InputAdapter - Exchange [c_current_addr_sk,c_customer_sk] #3 + Exchange [c_customer_sk,c_current_addr_sk] #3 WholeStageCodegen (6) - HashAggregate [c_current_addr_sk,c_customer_sk] - Project [c_current_addr_sk,c_customer_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + BroadcastHashJoin [customer_sk,c_customer_sk] Project [customer_sk] - BroadcastHashJoin [d_date_sk,sold_date_sk] - Project [customer_sk,sold_date_sk] - BroadcastHashJoin [i_item_sk,item_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] InputAdapter Union WholeStageCodegen (1) - Project [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] WholeStageCodegen (2) - Project [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -50,29 +50,29 @@ TakeOrderedAndProject [num_customers,segment,segment_base] Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) @@ -91,7 +91,7 @@ TakeOrderedAndProject [num_customers,segment,segment_base] BroadcastExchange #10 WholeStageCodegen (10) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] Subquery #1 WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] @@ -100,10 +100,10 @@ TakeOrderedAndProject [num_customers,segment,segment_base] WholeStageCodegen (1) HashAggregate [(d_month_seq + 1)] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] Subquery #2 WholeStageCodegen (2) HashAggregate [(d_month_seq + 3)] @@ -112,10 +112,10 @@ TakeOrderedAndProject [num_customers,segment,segment_base] WholeStageCodegen (1) HashAggregate [(d_month_seq + 3)] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt index d3ab9692efb60..b0d0e0d809441 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,ext_price] +TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_brand_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index b20fb74817e84..bb2ef8e82515c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [brand,brand_id,ext_price] +TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (4) - HashAggregate [i_brand,i_brand_id,sum] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id] #1 WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [i_brand,i_brand_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index f8bc60fffb502..5825c6f6e757a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -1,41 +1,41 @@ -TakeOrderedAndProject [i_item_id,total_sales] +TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (20) - HashAggregate [i_item_id,isEmpty,sum] [isEmpty,sum,sum(total_sales),total_sales] + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 WholeStageCodegen (5) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_item_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] @@ -46,7 +46,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -54,48 +54,48 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_color] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_item_id] + Scan parquet default.item [i_item_id,i_color] WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(cs_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #7 WholeStageCodegen (11) - HashAggregate [cs_ext_sales_price,i_item_id] [sum,sum] + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk] - BroadcastHashJoin [ca_address_sk,cs_bill_addr_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_item_id,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index f8bc60fffb502..5825c6f6e757a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,41 +1,41 @@ -TakeOrderedAndProject [i_item_id,total_sales] +TakeOrderedAndProject [total_sales,i_item_id] WholeStageCodegen (20) - HashAggregate [i_item_id,isEmpty,sum] [isEmpty,sum,sum(total_sales),total_sales] + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 WholeStageCodegen (5) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_item_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] @@ -46,7 +46,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -54,48 +54,48 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_color] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_item_id] + Scan parquet default.item [i_item_id,i_color] WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(cs_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #7 WholeStageCodegen (11) - HashAggregate [cs_ext_sales_price,i_item_id] [sum,sum] + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk] - BroadcastHashJoin [ca_address_sk,cs_bill_addr_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_item_id,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt index ba6b65472229a..852e1b085054d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/simplified.txt @@ -1,39 +1,39 @@ -TakeOrderedAndProject [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,nsum,psum,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (37) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,sum_sales,sum_sales,sum_sales] - SortMergeJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (25) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,rn,sum_sales,sum_sales] - SortMergeJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (13) - Sort [cc_name,i_brand,i_category,rn] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [cc_name,i_brand,i_category,rn] #1 + Exchange [i_category,i_brand,cc_name,rn] #1 WholeStageCodegen (12) - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (11) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #2 + Exchange [i_category,i_brand,cc_name] #2 WholeStageCodegen (10) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,sum_sales] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,cc_name,d_year,i_brand,i_category] + Window [_w0,i_category,i_brand,cc_name,d_year] WholeStageCodegen (9) - Sort [cc_name,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year] InputAdapter - Exchange [cc_name,d_year,i_brand,i_category] #3 + Exchange [i_category,i_brand,cc_name,d_year] #3 WholeStageCodegen (8) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [_w0,sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [cc_name,d_moy,d_year,i_brand,i_category] #4 + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #4 WholeStageCodegen (7) - HashAggregate [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] [sum,sum] - Project [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) @@ -41,21 +41,21 @@ TakeOrderedAndProject [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category InputAdapter Exchange [cs_item_sk] #5 WholeStageCodegen (3) - Project [cc_name,cs_item_sk,cs_sales_price,d_moy,d_year] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_moy,d_year] + Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_call_center_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_item_sk,cs_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) @@ -69,39 +69,39 @@ TakeOrderedAndProject [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category InputAdapter Exchange [i_item_sk] #8 WholeStageCodegen (5) - Filter [i_brand,i_category,i_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (24) - Sort [cc_name,i_brand,i_category,rn] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [cc_name,i_brand,i_category,rn] #9 + Exchange [i_category,i_brand,cc_name,rn] #9 WholeStageCodegen (23) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (22) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #10 + Exchange [i_category,i_brand,cc_name] #10 WholeStageCodegen (21) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum] #4 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #4 InputAdapter WholeStageCodegen (36) - Sort [cc_name,i_brand,i_category,rn] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [cc_name,i_brand,i_category,rn] #11 + Exchange [i_category,i_brand,cc_name,rn] #11 WholeStageCodegen (35) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (34) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum_sales] #10 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index 94e8f21d46a8a..432441bb788e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,54 +1,54 @@ -TakeOrderedAndProject [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,nsum,psum,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] WholeStageCodegen (23) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,rn,sum_sales,sum_sales] - BroadcastHashJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (8) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #1 + Exchange [i_category,i_brand,cc_name] #1 WholeStageCodegen (7) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,sum_sales] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,cc_name,d_year,i_brand,i_category] + Window [_w0,i_category,i_brand,cc_name,d_year] WholeStageCodegen (6) - Sort [cc_name,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year] InputAdapter - Exchange [cc_name,d_year,i_brand,i_category] #2 + Exchange [i_category,i_brand,cc_name,d_year] #2 WholeStageCodegen (5) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [_w0,sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [cc_name,d_moy,d_year,i_brand,i_category] #3 + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3 WholeStageCodegen (4) - HashAggregate [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] [sum,sum] - Project [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_sales_price,d_moy,d_year,i_brand,i_category] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_call_center_sk,cs_sales_price,cs_sold_date_sk,i_brand,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [i_brand,i_category,i_item_sk] + Project [i_brand,i_category,cs_sold_date_sk,cs_call_center_sk,cs_sales_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [cs_call_center_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_item_sk,cs_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -59,26 +59,26 @@ TakeOrderedAndProject [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category InputAdapter BroadcastExchange #7 WholeStageCodegen (15) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (14) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #8 + Exchange [i_category,i_brand,cc_name] #8 WholeStageCodegen (13) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum] #3 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter BroadcastExchange #9 WholeStageCodegen (22) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (21) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum_sales] #8 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 8498e4397271f..583dedd279e61 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -314,7 +314,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id= (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true)) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] @@ -433,7 +433,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id= (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true)) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt index 0b846774d3c97..fdd4ca9c3bee9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/simplified.txt @@ -1,23 +1,23 @@ -TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_dev,ws_item_rev] +TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] WholeStageCodegen (15) - Project [cs_item_rev,item_id,ss_item_rev,ws_item_rev] - BroadcastHashJoin [cs_item_rev,item_id,item_id,ss_item_rev,ws_item_rev] - Project [cs_item_rev,item_id,ss_item_rev] - BroadcastHashJoin [cs_item_rev,item_id,item_id,ss_item_rev] + Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] + Project [item_id,ss_item_rev,cs_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [item_id,ss_item_rev,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (4) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_item_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) @@ -26,7 +26,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -45,27 +45,27 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (9) Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [cs_item_rev,item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] InputAdapter Exchange [i_item_id] #6 WholeStageCodegen (8) - HashAggregate [cs_ext_sales_price,i_item_id] [sum,sum] + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) @@ -74,7 +74,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #8 WholeStageCodegen (5) @@ -85,24 +85,24 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #4 + ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter BroadcastExchange #9 WholeStageCodegen (14) Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [item_id,sum,sum(UnscaledValue(ws_ext_sales_price)),ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter Exchange [i_item_id] #10 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_item_id,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #11 WholeStageCodegen (11) @@ -111,7 +111,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 WholeStageCodegen (10) @@ -122,4 +122,4 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ InputAdapter Scan parquet default.date_dim [d_date,d_week_seq] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #4 + ReusedExchange [i_item_sk,i_item_id] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index 11295424fc7e3..5081efe9476cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_dev,ws_item_rev] +TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] WholeStageCodegen (15) - Project [cs_item_rev,item_id,ss_item_rev,ws_item_rev] - BroadcastHashJoin [cs_item_rev,item_id,item_id,ss_item_rev,ws_item_rev] - Project [cs_item_rev,item_id,ss_item_rev] - BroadcastHashJoin [cs_item_rev,item_id,item_id,ss_item_rev] + Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] + Project [item_id,ss_item_rev,cs_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [item_id,ss_item_rev,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (4) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_item_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_item_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) @@ -33,7 +33,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -53,21 +53,21 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #5 WholeStageCodegen (9) Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [cs_item_rev,item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] InputAdapter Exchange [i_item_id] #6 WholeStageCodegen (8) - HashAggregate [cs_ext_sales_price,i_item_id] [sum,sum] + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter BroadcastExchange #7 WholeStageCodegen (7) @@ -76,7 +76,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) @@ -90,21 +90,21 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ BroadcastExchange #9 WholeStageCodegen (14) Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [item_id,sum,sum(UnscaledValue(ws_ext_sales_price)),ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter Exchange [i_item_id] #10 WholeStageCodegen (13) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_item_id,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_item_id,ws_ext_sales_price,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter BroadcastExchange #11 WholeStageCodegen (12) @@ -113,7 +113,7 @@ TakeOrderedAndProject [average,cs_dev,cs_item_rev,item_id,ss_dev,ss_item_rev,ws_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #12 WholeStageCodegen (11) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt index de4e9930a2d38..a44ae96569f35 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_sales1 / sat_sales2),(sun_sales1 / sun_sales2),(thu_sales1 / thu_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),d_week_seq1,s_store_id1,s_store_name1] +TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) - Project [d_week_seq1,fri_sales1,fri_sales2,mon_sales1,mon_sales2,s_store_id1,s_store_name1,sat_sales1,sat_sales2,sun_sales1,sun_sales2,thu_sales1,thu_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2,s_store_id1,s_store_id2] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #1 WholeStageCodegen (2) - HashAggregate [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -42,20 +42,20 @@ TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_ InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index de4e9930a2d38..a44ae96569f35 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_sales1 / sat_sales2),(sun_sales1 / sun_sales2),(thu_sales1 / thu_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),d_week_seq1,s_store_id1,s_store_name1] +TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] WholeStageCodegen (10) - Project [d_week_seq1,fri_sales1,fri_sales2,mon_sales1,mon_sales2,s_store_id1,s_store_name1,sat_sales1,sat_sales2,sun_sales1,sun_sales2,thu_sales1,thu_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2,s_store_id1,s_store_id2] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,s_store_name,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #1 WholeStageCodegen (2) - HashAggregate [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,d_week_seq,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [d_date_sk,d_week_seq] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_day_name,d_week_seq] + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) @@ -42,20 +42,20 @@ TakeOrderedAndProject [(fri_sales1 / fri_sales2),(mon_sales1 / mon_sales2),(sat_ InputAdapter BroadcastExchange #5 WholeStageCodegen (9) - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,fri_sales,mon_sales,s_store_id,sat_sales,sun_sales,thu_sales,tue_sales,wed_sales] - BroadcastHashJoin [s_store_sk,ss_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [fri_sales,mon_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sun_sales,thu_sales,tue_sales,wed_sales] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN ss_sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN ss_sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) - Filter [s_store_id,s_store_sk] + Filter [s_store_sk,s_store_id] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index f97e8686c4421..dcebba331afb3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (14) - Project [cnt,state] + Project [state,cnt] Filter [count(1)] - HashAggregate [ca_state,count] [cnt,count,count(1),count(1),state] + HashAggregate [ca_state,count] [count(1),state,cnt,count(1),count] InputAdapter Exchange [ca_state] #1 WholeStageCodegen (13) HashAggregate [ca_state] [count,count] Project [ca_state] - SortMergeJoin [c_customer_sk,ss_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] @@ -16,18 +16,18 @@ TakeOrderedAndProject [cnt,state] Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Project [ss_customer_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_customer_sk,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] Subquery #1 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -36,10 +36,10 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) HashAggregate [d_month_seq] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -47,32 +47,32 @@ TakeOrderedAndProject [cnt,state] BroadcastExchange #5 WholeStageCodegen (4) Project [i_item_sk] - Filter [avg(i_current_price),i_current_price] + Filter [i_current_price,avg(i_current_price)] BroadcastHashJoin [i_category,i_category] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - HashAggregate [count,i_category,sum] [avg(UnscaledValue(i_current_price)),avg(i_current_price),count,i_category,sum] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #7 WholeStageCodegen (2) - HashAggregate [i_category,i_current_price] [count,count,sum,sum] + HashAggregate [i_category,i_current_price] [sum,count,sum,count] Filter [i_category] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price] + Scan parquet default.item [i_current_price,i_category] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #8 WholeStageCodegen (11) - Project [c_customer_sk,ca_state] - SortMergeJoin [c_current_addr_sk,ca_address_sk] + Project [ca_state,c_customer_sk] + SortMergeJoin [ca_address_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (8) Sort [ca_address_sk] @@ -92,4 +92,4 @@ TakeOrderedAndProject [cnt,state] Filter [c_current_addr_sk,c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 9b39a0cb65f56..97ba162948761 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (8) - Project [cnt,state] + Project [state,cnt] Filter [count(1)] - HashAggregate [ca_state,count] [cnt,count,count(1),count(1),state] + HashAggregate [ca_state,count] [count(1),state,cnt,count(1),count] InputAdapter Exchange [ca_state] #1 WholeStageCodegen (7) HashAggregate [ca_state] [count,count] Project [ca_state] - BroadcastHashJoin [i_item_sk,ss_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ca_state,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_sold_date_sk,ss_item_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [c_customer_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] Filter [ca_address_sk] ColumnarToRow InputAdapter @@ -25,19 +25,19 @@ TakeOrderedAndProject [cnt,state] Filter [c_current_addr_sk,c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] Subquery #1 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -46,10 +46,10 @@ TakeOrderedAndProject [cnt,state] WholeStageCodegen (1) HashAggregate [d_month_seq] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -57,21 +57,21 @@ TakeOrderedAndProject [cnt,state] BroadcastExchange #6 WholeStageCodegen (6) Project [i_item_sk] - Filter [avg(i_current_price),i_current_price] + Filter [i_current_price,avg(i_current_price)] BroadcastHashJoin [i_category,i_category] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [count,i_category,sum] [avg(UnscaledValue(i_current_price)),avg(i_current_price),count,i_category,sum] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) - HashAggregate [i_category,i_current_price] [count,count,sum,sum] + HashAggregate [i_category,i_current_price] [sum,count,sum,count] Filter [i_category] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price] + Scan parquet default.item [i_current_price,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index d49bf8669bea5..fb9e4e50775dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (20) - HashAggregate [i_item_id,isEmpty,sum] [isEmpty,sum,sum(total_sales),total_sales] + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 WholeStageCodegen (5) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_item_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] @@ -46,7 +46,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -54,48 +54,48 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_category] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_id] + Scan parquet default.item [i_item_id,i_category] WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(cs_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #7 WholeStageCodegen (11) - HashAggregate [cs_ext_sales_price,i_item_id] [sum,sum] + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk] - BroadcastHashJoin [ca_address_sk,cs_bill_addr_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_item_id,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index d49bf8669bea5..fb9e4e50775dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [i_item_id,total_sales] WholeStageCodegen (20) - HashAggregate [i_item_id,isEmpty,sum] [isEmpty,sum,sum(total_sales),total_sales] + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (19) - HashAggregate [i_item_id,total_sales] [isEmpty,isEmpty,sum,sum] + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 WholeStageCodegen (5) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_item_id,ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_item_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] @@ -46,7 +46,7 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -54,48 +54,48 @@ TakeOrderedAndProject [i_item_id,total_sales] Filter [i_category] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_id] + Scan parquet default.item [i_item_id,i_category] WholeStageCodegen (12) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(cs_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #7 WholeStageCodegen (11) - HashAggregate [cs_ext_sales_price,i_item_id] [sum,sum] + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk] - BroadcastHashJoin [ca_address_sk,cs_bill_addr_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_addr_sk,cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [i_item_id,sum] [sum,sum(UnscaledValue(ws_ext_sales_price)),total_sales] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_item_id,ws_ext_sales_price] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [ca_address_sk,ws_bill_addr_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt index 4fa1b4dd0a240..039ccb1aa18cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20))),promotions,total] +TakeOrderedAndProject [promotions,total,(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))] WholeStageCodegen (16) Project [promotions,total] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (8) - HashAggregate [sum] [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] InputAdapter Exchange #1 WholeStageCodegen (7) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_promo_sk,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_promo_sk,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_promo_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -38,7 +38,7 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -46,7 +46,7 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -54,45 +54,45 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( Filter [s_gmt_offset,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_gmt_offset,s_store_sk] + Scan parquet default.store [s_store_sk,s_gmt_offset] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) Project [c_customer_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] BroadcastExchange #8 WholeStageCodegen (15) - HashAggregate [sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total] + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] InputAdapter Exchange #9 WholeStageCodegen (14) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Project [ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter ReusedExchange [d_date_sk] #2 - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_store_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter ReusedExchange [i_item_sk] #3 InputAdapter @@ -101,10 +101,10 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( BroadcastExchange #10 WholeStageCodegen (13) Project [c_customer_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] InputAdapter ReusedExchange [ca_address_sk] #7 - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 438b46afbd687..da75651673cfe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20))),promotions,total] +TakeOrderedAndProject [promotions,total,(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL(15,4)) / CAST(CAST(total AS DECIMAL(15,4)) AS DECIMAL(15,4))) AS DECIMAL(35,20)) * CAST(CAST(100 AS DECIMAL(3,0)) AS DECIMAL(35,20)))] WholeStageCodegen (16) Project [promotions,total] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (8) - HashAggregate [sum] [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] InputAdapter Exchange #1 WholeStageCodegen (7) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_promo_sk,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_customer_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_promo_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -32,7 +32,7 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( Filter [s_gmt_offset,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_gmt_offset,s_store_sk] + Scan parquet default.store [s_store_sk,s_gmt_offset] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -40,27 +40,27 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( Filter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] @@ -71,34 +71,34 @@ TakeOrderedAndProject [(CAST((CAST(CAST(promotions AS DECIMAL(15,4)) AS DECIMAL( Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_category] BroadcastExchange #8 WholeStageCodegen (15) - HashAggregate [sum] [sum,sum(UnscaledValue(ss_ext_sales_price)),total] + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] InputAdapter Exchange #9 WholeStageCodegen (14) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_sold_date_sk,ss_customer_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter ReusedExchange [s_store_sk] #2 InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter - ReusedExchange [c_current_addr_sk,c_customer_sk] #5 + ReusedExchange [c_customer_sk,c_current_addr_sk] #5 InputAdapter ReusedExchange [ca_address_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt index 0cc558cd526f8..59cfc4b7b249a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/simplified.txt @@ -1,37 +1,37 @@ -TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sm_type,substr(w_warehouse_name, 1, 20),web_name] +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) - HashAggregate [sm_type,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,web_name] [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint))] + HashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint)),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter - Exchange [sm_type,substr(w_warehouse_name, 1, 20),web_name] #1 + Exchange [substr(w_warehouse_name, 1, 20),sm_type,web_name] #1 WholeStageCodegen (5) - HashAggregate [sm_type,w_warehouse_name,web_name,ws_ship_date_sk,ws_sold_date_sk] [substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sm_type,w_warehouse_name,web_name,ws_ship_date_sk,ws_sold_date_sk] - BroadcastHashJoin [w_warehouse_sk,ws_warehouse_sk] - Project [sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk,ws_warehouse_sk] - BroadcastHashJoin [sm_ship_mode_sk,ws_ship_mode_sk] - Project [web_name,ws_ship_date_sk,ws_ship_mode_sk,ws_sold_date_sk,ws_warehouse_sk] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [ws_ship_date_sk,ws_ship_mode_sk,ws_sold_date_sk,ws_warehouse_sk,ws_web_site_sk] + HashAggregate [w_warehouse_name,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] + Project [ws_sold_date_sk,ws_ship_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Project [ws_sold_date_sk,ws_ship_date_sk,ws_warehouse_sk,web_name,sm_type] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_sold_date_sk,ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,web_name] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] BroadcastHashJoin [d_date_sk,ws_ship_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] - Filter [ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_web_site_sk] + Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_mode_sk,ws_sold_date_sk,ws_warehouse_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_name,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -45,4 +45,4 @@ TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index dda342f6e7c96..803326b2afd30 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -1,29 +1,29 @@ -TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sm_type,substr(w_warehouse_name, 1, 20),web_name] +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) - HashAggregate [sm_type,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,web_name] [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint))] + HashAggregate [substr(w_warehouse_name, 1, 20),sm_type,web_name,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint)),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter - Exchange [sm_type,substr(w_warehouse_name, 1, 20),web_name] #1 + Exchange [substr(w_warehouse_name, 1, 20),sm_type,web_name] #1 WholeStageCodegen (5) - HashAggregate [sm_type,w_warehouse_name,web_name,ws_ship_date_sk,ws_sold_date_sk] [substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sm_type,w_warehouse_name,web_name,ws_ship_date_sk,ws_sold_date_sk] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] - Project [sm_type,w_warehouse_name,web_name,ws_ship_date_sk,ws_sold_date_sk] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [sm_type,w_warehouse_name,ws_ship_date_sk,ws_sold_date_sk,ws_web_site_sk] - BroadcastHashJoin [sm_ship_mode_sk,ws_ship_mode_sk] - Project [w_warehouse_name,ws_ship_date_sk,ws_ship_mode_sk,ws_sold_date_sk,ws_web_site_sk] - BroadcastHashJoin [w_warehouse_sk,ws_warehouse_sk] - Filter [ws_ship_date_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_web_site_sk] + HashAggregate [w_warehouse_name,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] + Project [ws_sold_date_sk,ws_ship_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ship_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,w_warehouse_name] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Filter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ship_date_sk,ws_ship_mode_sk,ws_sold_date_sk,ws_warehouse_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -37,12 +37,12 @@ TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_name,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/simplified.txt index 99d321b52694a..9ae026f0f750d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (7) - Project [avg_monthly_sales,i_manager_id,sum_sales] + Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] @@ -9,29 +9,29 @@ TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] InputAdapter Exchange [i_manager_id] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_manager_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_manager_id] #2 + Exchange [i_manager_id,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_manager_id,ss_sales_price] [sum,sum] - Project [d_moy,i_manager_id,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manager_id,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_manager_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_sold_date_sk,ss_sales_price] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [i_item_sk,i_manager_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -43,7 +43,7 @@ TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 7d1b1e0ae9c41..7272c01abbaff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] WholeStageCodegen (7) - Project [avg_monthly_sales,i_manager_id,sum_sales] + Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] @@ -9,34 +9,34 @@ TakeOrderedAndProject [avg_monthly_sales,i_manager_id,sum_sales] InputAdapter Exchange [i_manager_id] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_manager_id,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_manager_id] #2 + Exchange [i_manager_id,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_manager_id,ss_sales_price] [sum,sum] - Project [d_moy,i_manager_id,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,i_manager_id,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_manager_id,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] Project [i_item_sk,i_manager_id] - Filter [i_brand,i_category,i_class,i_item_sk] + Filter [i_category,i_class,i_brand,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/simplified.txt index cf75c1b7685f5..74ddaf3999a8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/simplified.txt @@ -1,28 +1,28 @@ WholeStageCodegen (88) - Sort [cnt,product_name,store_name] + Sort [product_name,store_name,cnt] InputAdapter - Exchange [cnt,product_name,store_name] #1 + Exchange [product_name,store_name,cnt] #1 WholeStageCodegen (87) - Project [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,cnt,product_name,s1,s1,s2,s2,s3,s3,store_name,store_zip,syear,syear] - SortMergeJoin [cnt,cnt,item_sk,item_sk,store_name,store_name,store_zip,store_zip] + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter WholeStageCodegen (43) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #2 WholeStageCodegen (42) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,count,count(1),item_sk,product_name,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #3 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #3 WholeStageCodegen (41) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (35) @@ -30,21 +30,21 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_addr_sk] #4 WholeStageCodegen (34) - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (31) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #5 WholeStageCodegen (30) - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] InputAdapter WholeStageCodegen (24) @@ -52,44 +52,44 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_cdemo_sk] #6 WholeStageCodegen (23) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [cd_demo_sk,ss_cdemo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + SortMergeJoin [ss_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (20) Sort [ss_cdemo_sk] InputAdapter Exchange [ss_cdemo_sk] #7 WholeStageCodegen (19) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (14) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (13) - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (2) Sort [ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_item_sk,ss_ticket_number] #9 WholeStageCodegen (1) - Filter [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] @@ -104,14 +104,14 @@ WholeStageCodegen (88) BroadcastExchange #11 WholeStageCodegen (10) Project [cs_item_sk] - Filter [sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price)] - HashAggregate [cs_item_sk,isEmpty,sum,sum] [isEmpty,sum,sum,sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(UnscaledValue(cs_ext_list_price)),sum(cs_ext_list_price)] + Filter [sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true))] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #12 WholeStageCodegen (9) - HashAggregate [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] [isEmpty,isEmpty,sum,sum,sum,sum] - Project [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter WholeStageCodegen (6) Sort [cs_item_sk,cs_order_number] @@ -121,7 +121,7 @@ WholeStageCodegen (88) Filter [cs_item_sk,cs_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_list_price,cs_item_sk,cs_order_number] + Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price] InputAdapter WholeStageCodegen (8) Sort [cr_item_sk,cr_order_number] @@ -135,27 +135,27 @@ WholeStageCodegen (88) InputAdapter BroadcastExchange #15 WholeStageCodegen (11) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #16 WholeStageCodegen (12) - Filter [s_store_name,s_store_sk,s_zip] + Filter [s_store_sk,s_store_name,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (16) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #17 WholeStageCodegen (15) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #18 WholeStageCodegen (17) @@ -205,12 +205,12 @@ WholeStageCodegen (88) Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter WholeStageCodegen (37) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #22 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #22 InputAdapter BroadcastExchange #23 WholeStageCodegen (38) @@ -224,28 +224,28 @@ WholeStageCodegen (88) BroadcastExchange #24 WholeStageCodegen (40) Project [i_item_sk,i_product_name] - Filter [i_color,i_current_price,i_item_sk] + Filter [i_current_price,i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (86) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #25 WholeStageCodegen (85) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [cnt,count,count(1),item_sk,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #26 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #26 WholeStageCodegen (84) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (78) @@ -253,21 +253,21 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_addr_sk] #27 WholeStageCodegen (77) - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (74) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #28 WholeStageCodegen (73) - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] InputAdapter WholeStageCodegen (67) @@ -275,39 +275,39 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_cdemo_sk] #29 WholeStageCodegen (66) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [cd_demo_sk,ss_cdemo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + SortMergeJoin [ss_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (63) Sort [ss_cdemo_sk] InputAdapter Exchange [ss_cdemo_sk] #30 WholeStageCodegen (62) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (57) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #31 WholeStageCodegen (56) - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (45) Sort [ss_item_sk,ss_ticket_number] InputAdapter - ReusedExchange [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] #9 + ReusedExchange [ss_sold_date_sk,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] #9 InputAdapter WholeStageCodegen (47) Sort [sr_item_sk,sr_ticket_number] @@ -318,17 +318,17 @@ WholeStageCodegen (88) InputAdapter BroadcastExchange #32 WholeStageCodegen (54) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [s_store_name,s_store_sk,s_zip] #16 + ReusedExchange [s_store_sk,s_store_name,s_zip] #16 InputAdapter WholeStageCodegen (59) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] #17 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #17 InputAdapter ReusedExchange [d_date_sk,d_year] #18 InputAdapter @@ -353,12 +353,12 @@ WholeStageCodegen (88) WholeStageCodegen (76) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #22 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #22 InputAdapter WholeStageCodegen (80) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #22 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #22 InputAdapter ReusedExchange [ib_income_band_sk] #23 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 5b6dac01866b7..0e658ff99f2ed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -323,7 +323,7 @@ Input [13]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, s Output [3]: [s_store_sk#40, s_store_name#41, s_zip#42] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip), IsNotNull(s_store_name)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct (34) ColumnarToRow [codegen id : 6] @@ -331,7 +331,7 @@ Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] (35) Filter [codegen id : 6] Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] -Condition : ((isnotnull(s_store_sk#40) AND isnotnull(s_zip#42)) AND isnotnull(s_store_name#41)) +Condition : ((isnotnull(s_store_sk#40) AND isnotnull(s_store_name#41)) AND isnotnull(s_zip#42)) (36) BroadcastExchange Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index 78b316fcc2195..4c40a359cf4af 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -1,53 +1,53 @@ WholeStageCodegen (43) - Sort [cnt,product_name,store_name] + Sort [product_name,store_name,cnt] InputAdapter - Exchange [cnt,product_name,store_name] #1 + Exchange [product_name,store_name,cnt] #1 WholeStageCodegen (42) - Project [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,cnt,product_name,s1,s1,s2,s2,s3,s3,store_name,store_zip,syear,syear] - BroadcastHashJoin [cnt,cnt,item_sk,item_sk,store_name,store_name,store_zip,store_zip] - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,count,count(1),item_sk,product_name,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + BroadcastHashJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #2 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #2 WholeStageCodegen (20) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -59,18 +59,18 @@ WholeStageCodegen (43) BroadcastExchange #4 WholeStageCodegen (4) Project [cs_item_sk] - Filter [sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price)] - HashAggregate [cs_item_sk,isEmpty,sum,sum] [isEmpty,sum,sum,sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(UnscaledValue(cs_ext_list_price)),sum(cs_ext_list_price)] + Filter [sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true))] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #5 WholeStageCodegen (3) - HashAggregate [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] [isEmpty,isEmpty,sum,sum,sum,sum] - Project [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] Filter [cs_item_sk,cs_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_list_price,cs_item_sk,cs_order_number] + Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -81,24 +81,24 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Filter [s_store_name,s_store_sk,s_zip] + Filter [s_store_sk,s_store_name,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -139,9 +139,9 @@ WholeStageCodegen (43) Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 InputAdapter BroadcastExchange #15 WholeStageCodegen (17) @@ -155,56 +155,56 @@ WholeStageCodegen (43) BroadcastExchange #16 WholeStageCodegen (19) Project [i_item_sk,i_product_name] - Filter [i_color,i_current_price,i_item_sk] + Filter [i_current_price,i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter BroadcastExchange #17 WholeStageCodegen (41) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [cnt,count,count(1),item_sk,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #18 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #18 WholeStageCodegen (40) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter ReusedExchange [sr_item_sk,sr_ticket_number] #3 InputAdapter @@ -212,14 +212,14 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #19 WholeStageCodegen (25) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [s_store_name,s_store_sk,s_zip] #8 + ReusedExchange [s_store_sk,s_store_name,s_zip] #8 InputAdapter - ReusedExchange [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] #9 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #10 InputAdapter @@ -235,9 +235,9 @@ WholeStageCodegen (43) InputAdapter ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 InputAdapter ReusedExchange [ib_income_band_sk] #15 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt index 0d9bf297d5ae0..9fc7b97cf6240 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt @@ -1,56 +1,56 @@ -TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] +TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (12) - Project [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (9) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #1 WholeStageCodegen (8) - Project [revenue,s_store_name,ss_item_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [revenue,ss_item_sk,ss_store_sk] - BroadcastHashJoin [ave,revenue,ss_store_sk,ss_store_sk] + Project [ss_item_sk,revenue,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_item_sk,revenue] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Filter [revenue] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #2 + Exchange [ss_store_sk,ss_item_sk] #2 WholeStageCodegen (2) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (6) - HashAggregate [count,ss_store_sk,sum] [ave,avg(revenue),count,sum] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #5 WholeStageCodegen (5) - HashAggregate [revenue,ss_store_sk] [count,count,sum,sum] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #6 + Exchange [ss_store_sk,ss_item_sk] #6 WholeStageCodegen (4) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -59,7 +59,7 @@ TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,reve Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] InputAdapter WholeStageCodegen (11) Sort [i_item_sk] @@ -69,4 +69,4 @@ TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,reve Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_current_price,i_item_desc,i_item_sk,i_wholesale_cost] + Scan parquet default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 77f6cc15c8210..a4b468ffeb0c9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] +TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] WholeStageCodegen (9) - Project [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name] - BroadcastHashJoin [ave,revenue,ss_store_sk,ss_store_sk] - Project [i_brand,i_current_price,i_item_desc,i_wholesale_cost,revenue,s_store_name,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [revenue,s_store_name,ss_item_sk,ss_store_sk] + Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [s_store_name,ss_store_sk,ss_item_sk,revenue] BroadcastHashJoin [s_store_sk,ss_store_sk] Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 WholeStageCodegen (3) Filter [revenue] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #2 + Exchange [ss_store_sk,ss_item_sk] #2 WholeStageCodegen (2) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -39,25 +39,25 @@ TakeOrderedAndProject [i_brand,i_current_price,i_item_desc,i_wholesale_cost,reve Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_current_price,i_item_desc,i_item_sk,i_wholesale_cost] + Scan parquet default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - HashAggregate [count,ss_store_sk,sum] [ave,avg(revenue),count,sum] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter Exchange [ss_store_sk] #6 WholeStageCodegen (7) - HashAggregate [revenue,ss_store_sk] [count,count,sum,sum] - HashAggregate [ss_item_sk,ss_store_sk,sum] [revenue,sum,sum(UnscaledValue(ss_sales_price))] + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_item_sk,ss_store_sk] #7 + Exchange [ss_store_sk,ss_item_sk] #7 WholeStageCodegen (6) - HashAggregate [ss_item_sk,ss_sales_price,ss_store_sk] [sum,sum] - Project [ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt index 488b1c27b8987..465d269a847c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt @@ -1,25 +1,25 @@ -TakeOrderedAndProject [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales,aug_sales_per_sq_foot,dec_net,dec_sales,dec_sales_per_sq_foot,feb_net,feb_sales,feb_sales_per_sq_foot,jan_net,jan_sales,jan_sales_per_sq_foot,jul_net,jul_sales,jul_sales_per_sq_foot,jun_net,jun_sales,jun_sales_per_sq_foot,mar_net,mar_sales,mar_sales_per_sq_foot,may_net,may_sales,may_sales_per_sq_foot,nov_net,nov_sales,nov_sales_per_sq_foot,oct_net,oct_sales,oct_sales_per_sq_foot,sep_net,sep_sales,sep_sales_per_sq_foot,ship_carriers,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] +TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] WholeStageCodegen (14) - HashAggregate [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,ship_carriers,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales,aug_sales_per_sq_foot,dec_net,dec_sales,dec_sales_per_sq_foot,feb_net,feb_sales,feb_sales_per_sq_foot,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,jan_net,jan_sales,jan_sales_per_sq_foot,jul_net,jul_sales,jul_sales_per_sq_foot,jun_net,jun_sales,jun_sales_per_sq_foot,mar_net,mar_sales,mar_sales_per_sq_foot,may_net,may_sales,may_sales_per_sq_foot,nov_net,nov_sales,nov_sales_per_sq_foot,oct_net,oct_sales,oct_sales_per_sq_foot,sep_net,sep_sales,sep_sales_per_sq_foot,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum(CheckOverflow((promote_precision(apr_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(aug_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(dec_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(feb_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jan_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jul_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jun_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(mar_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(may_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(nov_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(oct_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(sep_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(apr_net),sum(apr_sales),sum(aug_net),sum(aug_sales),sum(dec_net),sum(dec_sales),sum(feb_net),sum(feb_sales),sum(jan_net),sum(jan_sales),sum(jul_net),sum(jul_sales),sum(jun_net),sum(jun_sales),sum(mar_net),sum(mar_sales),sum(may_net),sum(may_sales),sum(nov_net),sum(nov_sales),sum(oct_net),sum(oct_sales),sum(sep_net),sum(sep_sales)] + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum(CheckOverflow((promote_precision(jan_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(feb_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(mar_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(apr_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(may_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jun_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jul_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(aug_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(sep_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(oct_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(nov_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(dec_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ship_carriers,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] #1 + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 WholeStageCodegen (13) - HashAggregate [apr_net,apr_sales,aug_net,aug_sales,dec_net,dec_sales,feb_net,feb_sales,jan_net,jan_sales,jul_net,jul_sales,jun_net,jun_sales,mar_net,mar_sales,may_net,may_sales,nov_net,nov_sales,oct_net,oct_sales,sep_net,sep_sales,ship_carriers,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,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,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] InputAdapter Union WholeStageCodegen (6) - HashAggregate [d_year,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] [apr_net,apr_sales,aug_net,aug_sales,dec_net,dec_sales,feb_net,feb_sales,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,jan_net,jan_sales,jul_net,jul_sales,jun_net,jun_sales,mar_net,mar_sales,may_net,may_sales,nov_net,nov_sales,oct_net,oct_sales,sep_net,sep_sales,ship_carriers,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),year] + 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 CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) 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] InputAdapter - Exchange [d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] #2 + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 WholeStageCodegen (5) - HashAggregate [d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity] - BroadcastHashJoin [w_warehouse_sk,ws_warehouse_sk] - Project [d_moy,d_year,ws_ext_sales_price,ws_net_paid,ws_quantity,ws_warehouse_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_ext_sales_price,ws_net_paid,ws_quantity,ws_sold_date_sk,ws_warehouse_sk] - BroadcastHashJoin [t_time_sk,ws_sold_time_sk] - Project [ws_ext_sales_price,ws_net_paid,ws_quantity,ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk] + HashAggregate [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] [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,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] + Project [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] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Project [ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] BroadcastHashJoin [sm_ship_mode_sk,ws_ship_mode_sk] InputAdapter BroadcastExchange #3 @@ -28,11 +28,11 @@ TakeOrderedAndProject [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales Filter [sm_carrier,sm_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet default.ship_mode [sm_carrier,sm_ship_mode_sk] - Filter [ws_ship_mode_sk,ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk] + Scan parquet default.ship_mode [sm_ship_mode_sk,sm_carrier] + Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_paid,ws_quantity,ws_ship_mode_sk,ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -40,44 +40,44 @@ TakeOrderedAndProject [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales Filter [t_time,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_time,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_time] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sk,w_warehouse_sq_ft] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] WholeStageCodegen (12) - HashAggregate [d_year,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] [apr_net,apr_sales,aug_net,aug_sales,dec_net,dec_sales,feb_net,feb_sales,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,jan_net,jan_sales,jul_net,jul_sales,jun_net,jun_sales,mar_net,mar_sales,may_net,may_sales,nov_net,nov_sales,oct_net,oct_sales,sep_net,sep_sales,ship_carriers,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),year] + 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 CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) 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] InputAdapter - Exchange [d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] #7 + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 WholeStageCodegen (11) - HashAggregate [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] + HashAggregate [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] [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,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] + Project [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] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_warehouse_sk,d_moy,d_year] + Project [cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,d_year,d_moy] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_warehouse_sk] + Project [cs_sold_date_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] + BroadcastHashJoin [sm_ship_mode_sk,cs_ship_mode_sk] InputAdapter ReusedExchange [sm_ship_mode_sk] #3 - Filter [cs_ship_mode_sk,cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk] + Filter [cs_warehouse_sk,cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_ship_mode_sk,cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] InputAdapter ReusedExchange [t_time_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy,d_year] #5 + ReusedExchange [d_date_sk,d_year,d_moy] #5 InputAdapter - ReusedExchange [w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sk,w_warehouse_sq_ft] #6 + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 2bce0e80224d2..ac7379973630f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,44 +1,44 @@ -TakeOrderedAndProject [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales,aug_sales_per_sq_foot,dec_net,dec_sales,dec_sales_per_sq_foot,feb_net,feb_sales,feb_sales_per_sq_foot,jan_net,jan_sales,jan_sales_per_sq_foot,jul_net,jul_sales,jul_sales_per_sq_foot,jun_net,jun_sales,jun_sales_per_sq_foot,mar_net,mar_sales,mar_sales_per_sq_foot,may_net,may_sales,may_sales_per_sq_foot,nov_net,nov_sales,nov_sales_per_sq_foot,oct_net,oct_sales,oct_sales_per_sq_foot,sep_net,sep_sales,sep_sales_per_sq_foot,ship_carriers,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] +TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] WholeStageCodegen (14) - HashAggregate [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,ship_carriers,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales,aug_sales_per_sq_foot,dec_net,dec_sales,dec_sales_per_sq_foot,feb_net,feb_sales,feb_sales_per_sq_foot,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,jan_net,jan_sales,jan_sales_per_sq_foot,jul_net,jul_sales,jul_sales_per_sq_foot,jun_net,jun_sales,jun_sales_per_sq_foot,mar_net,mar_sales,mar_sales_per_sq_foot,may_net,may_sales,may_sales_per_sq_foot,nov_net,nov_sales,nov_sales_per_sq_foot,oct_net,oct_sales,oct_sales_per_sq_foot,sep_net,sep_sales,sep_sales_per_sq_foot,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum(CheckOverflow((promote_precision(apr_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(aug_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(dec_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(feb_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jan_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jul_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jun_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(mar_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(may_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(nov_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(oct_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(sep_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(apr_net),sum(apr_sales),sum(aug_net),sum(aug_sales),sum(dec_net),sum(dec_sales),sum(feb_net),sum(feb_sales),sum(jan_net),sum(jan_sales),sum(jul_net),sum(jul_sales),sum(jun_net),sum(jun_sales),sum(mar_net),sum(mar_sales),sum(may_net),sum(may_sales),sum(nov_net),sum(nov_sales),sum(oct_net),sum(oct_sales),sum(sep_net),sum(sep_sales)] + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum(CheckOverflow((promote_precision(jan_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(feb_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(mar_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(apr_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(may_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jun_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(jul_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(aug_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(sep_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(oct_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(nov_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(CheckOverflow((promote_precision(dec_sales) / promote_precision(cast(cast(w_warehouse_sq_ft as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ship_carriers,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] #1 + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 WholeStageCodegen (13) - HashAggregate [apr_net,apr_sales,aug_net,aug_sales,dec_net,dec_sales,feb_net,feb_sales,jan_net,jan_sales,jul_net,jul_sales,jun_net,jun_sales,mar_net,mar_sales,may_net,may_sales,nov_net,nov_sales,oct_net,oct_sales,sep_net,sep_sales,ship_carriers,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,year] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,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,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] InputAdapter Union WholeStageCodegen (6) - HashAggregate [d_year,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] [apr_net,apr_sales,aug_net,aug_sales,dec_net,dec_sales,feb_net,feb_sales,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,jan_net,jan_sales,jul_net,jul_sales,jun_net,jun_sales,mar_net,mar_sales,may_net,may_sales,nov_net,nov_sales,oct_net,oct_sales,sep_net,sep_sales,ship_carriers,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),year] + 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 CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(ws_net_paid as decimal(12,2))) * promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) 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] InputAdapter - Exchange [d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] #2 + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 WholeStageCodegen (5) - HashAggregate [d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity] - BroadcastHashJoin [sm_ship_mode_sk,ws_ship_mode_sk] - Project [d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity,ws_ship_mode_sk] - BroadcastHashJoin [t_time_sk,ws_sold_time_sk] - Project [d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity,ws_ship_mode_sk,ws_sold_time_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft,ws_ext_sales_price,ws_net_paid,ws_quantity,ws_ship_mode_sk,ws_sold_date_sk,ws_sold_time_sk] - BroadcastHashJoin [w_warehouse_sk,ws_warehouse_sk] - Filter [ws_ship_mode_sk,ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk] + HashAggregate [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] [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,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] + Project [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] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [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] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,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] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + Filter [ws_warehouse_sk,ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_paid,ws_quantity,ws_ship_mode_sk,ws_sold_date_sk,ws_sold_time_sk,ws_warehouse_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sk,w_warehouse_sq_ft] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -46,7 +46,7 @@ TakeOrderedAndProject [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales Filter [t_time,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_time,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_time] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) @@ -54,29 +54,29 @@ TakeOrderedAndProject [apr_net,apr_sales,apr_sales_per_sq_foot,aug_net,aug_sales Filter [sm_carrier,sm_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet default.ship_mode [sm_carrier,sm_ship_mode_sk] + Scan parquet default.ship_mode [sm_ship_mode_sk,sm_carrier] WholeStageCodegen (12) - HashAggregate [d_year,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] [apr_net,apr_sales,aug_net,aug_sales,dec_net,dec_sales,feb_net,feb_sales,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,jan_net,jan_sales,jul_net,jul_sales,jun_net,jun_sales,mar_net,mar_sales,may_net,may_sales,nov_net,nov_sales,oct_net,oct_sales,sep_net,sep_sales,ship_carriers,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),year] + 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 CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax as decimal(12,2))) * promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) 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] InputAdapter - Exchange [d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] #7 + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 WholeStageCodegen (11) - HashAggregate [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] + HashAggregate [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] [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,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] + Project [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] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_ship_mode_sk,d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] + Project [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] BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_ship_mode_sk,cs_sold_time_sk,d_moy,d_year,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_ship_mode_sk,cs_sold_date_sk,cs_sold_time_sk,w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sq_ft] + Project [cs_sold_date_sk,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] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_ship_mode_sk,cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk] + Filter [cs_warehouse_sk,cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_net_paid_inc_tax,cs_quantity,cs_sales_price,cs_ship_mode_sk,cs_sold_date_sk,cs_sold_time_sk,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax] InputAdapter - ReusedExchange [w_city,w_country,w_county,w_state,w_warehouse_name,w_warehouse_sk,w_warehouse_sq_ft] #3 + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #3 InputAdapter - ReusedExchange [d_date_sk,d_moy,d_year] #4 + ReusedExchange [d_date_sk,d_year,d_moy] #4 InputAdapter ReusedExchange [t_time_sk] #5 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt index bba681c658162..185701fbc38df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt @@ -1,50 +1,50 @@ -TakeOrderedAndProject [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,rk,s_store_id,sumsales] +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] WholeStageCodegen (10) Filter [rk] InputAdapter - Window [i_category,sumsales] + Window [sumsales,i_category] WholeStageCodegen (9) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 WholeStageCodegen (8) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,spark_grouping_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - Exchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,spark_grouping_id] #2 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 WholeStageCodegen (7) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,spark_grouping_id,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Expand [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] - Project [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #3 WholeStageCodegen (3) - Project [d_moy,d_qoy,d_year,s_store_id,ss_item_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_qoy,d_year,ss_item_sk,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Project [d_date_sk,d_moy,d_qoy,d_year] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_year,d_moy,d_qoy] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] @@ -54,4 +54,4 @@ TakeOrderedAndProject [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_n Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index faf21736c3f60..b343c2d02c4ee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,rk,s_store_id,sumsales] +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] WholeStageCodegen (7) Filter [rk] InputAdapter - Window [i_category,sumsales] + Window [sumsales,i_category] WholeStageCodegen (6) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 WholeStageCodegen (5) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,spark_grouping_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - Exchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,spark_grouping_id] #2 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 WholeStageCodegen (4) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,spark_grouping_id,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Expand [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] - Project [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_moy,d_qoy,d_year,s_store_id,ss_item_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_qoy,d_year,ss_item_sk,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [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] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date_sk,d_moy,d_qoy,d_year] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_year,d_moy,d_qoy] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt index 0108f810f7b49..64672d38be957 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/simplified.txt @@ -1,7 +1,7 @@ -TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] +TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] WholeStageCodegen (16) - Project [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] - SortMergeJoin [bought_city,c_customer_sk,ca_city,ss_customer_sk] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + SortMergeJoin [c_customer_sk,ss_customer_sk,ca_city,bought_city] InputAdapter WholeStageCodegen (6) Sort [c_customer_sk] @@ -16,10 +16,10 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri InputAdapter Exchange [c_current_addr_sk] #2 WholeStageCodegen (1) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter WholeStageCodegen (4) Sort [ca_address_sk] @@ -36,34 +36,34 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (14) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum,sum] [bought_city,extended_price,extended_tax,list_price,sum,sum,sum,sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_tax))] - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] [sum,sum,sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - SortMergeJoin [ca_address_sk,ss_addr_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (11) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #5 WholeStageCodegen (10) - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #6 WholeStageCodegen (7) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #7 WholeStageCodegen (8) @@ -71,12 +71,12 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #8 WholeStageCodegen (9) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index 4c2d24b06c709..819e5504ff673 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] +TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] WholeStageCodegen (8) - Project [bought_city,c_first_name,c_last_name,ca_city,extended_price,extended_tax,list_price,ss_ticket_number] - BroadcastHashJoin [bought_city,c_current_addr_sk,ca_address_sk,ca_city] - Project [bought_city,c_current_addr_sk,c_first_name,c_last_name,extended_price,extended_tax,list_price,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum,sum] [bought_city,extended_price,extended_tax,list_price,sum,sum,sum,sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_tax))] + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] + Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] InputAdapter - Exchange [ca_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #1 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 WholeStageCodegen (5) - HashAggregate [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] [sum,sum,sum,sum,sum,sum] - Project [ca_city,ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_addr_sk,ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_customer_sk,ss_ext_list_price,ss_ext_sales_price,ss_ext_tax,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -36,12 +36,12 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri Filter [s_city,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_store_sk] + Scan parquet default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -55,9 +55,9 @@ TakeOrderedAndProject [bought_city,c_first_name,c_last_name,ca_city,extended_pri InputAdapter BroadcastExchange #6 WholeStageCodegen (6) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt index 0d637a4674e69..beca1370f3716 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/simplified.txt @@ -1,18 +1,18 @@ -TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3] +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] WholeStageCodegen (15) - HashAggregate [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,count,count(1)] + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] InputAdapter - Exchange [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 WholeStageCodegen (14) - HashAggregate [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (13) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] SortMergeJoin [c_customer_sk,ws_bill_customer_sk] @@ -25,37 +25,37 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (4) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (8) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #6 WholeStageCodegen (7) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (11) @@ -68,18 +68,18 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #8 WholeStageCodegen (12) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index ca9400b9875ef..b21892546da4e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -82,7 +82,7 @@ Condition : isnotnull(ss_sold_date_sk#4) Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] @@ -90,7 +90,7 @@ Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (9) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : (((((isnotnull(d_moy#8) AND isnotnull(d_year#7)) AND (d_year#7 = 2001)) AND (d_moy#8 >= 4)) AND (d_moy#8 <= 6)) AND isnotnull(d_date_sk#6)) +Condition : (((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 >= 4)) AND (d_moy#8 <= 6)) AND isnotnull(d_date_sk#6)) (10) Project [codegen id : 1] Output [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 765e6a7524dea..9e34940da5160 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3] +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] WholeStageCodegen (10) - HashAggregate [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,count,count(1)] + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] InputAdapter - Exchange [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 WholeStageCodegen (9) - HashAggregate [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -53,14 +53,14 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 WholeStageCodegen (7) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -70,4 +70,4 @@ TakeOrderedAndProject [cd_credit_rating,cd_education_status,cd_gender,cd_marital Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt index 3fed561c34984..38576c0adde4b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt @@ -53,7 +53,7 @@ Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND is Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_marital_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_education_status#12)) AND isnotnull(cd_marital_status#11)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/simplified.txt index 533447867c5aa..d78afa9979777 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_sales_price)),avg(cast(ss_quantity as bigint)),count,count,count,count,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) - HashAggregate [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk] + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -32,12 +32,12 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [p_channel_email,p_channel_event,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_email,p_channel_event,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -47,4 +47,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 998c6b2b02218..24546c68f63c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -53,7 +53,7 @@ Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND is Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -61,7 +61,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_marital_status#11) AND isnotnull(cd_gender#10)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = M)) AND (cd_marital_status#11 = S)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index db56467a0218d..075c6d15c8407 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_sales_price)),avg(cast(ss_quantity as bigint)),count,count,count,count,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(ss_quantity as bigint)),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) - HashAggregate [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk] + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_promo_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -39,7 +39,7 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) @@ -47,4 +47,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,i_item_id] Filter [p_channel_email,p_channel_event,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_email,p_channel_event,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt index 1b14fd24d1aee..ae76771689231 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_sum] +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] WholeStageCodegen (12) - Project [lochierarchy,rank_within_parent,s_county,s_state,total_sum] + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter - Window [_w1,_w2,_w3] + Window [_w3,_w1,_w2] WholeStageCodegen (11) Sort [_w1,_w2,_w3] InputAdapter Exchange [_w1,_w2] #1 WholeStageCodegen (10) - HashAggregate [s_county,s_state,spark_grouping_id,sum] [_w1,_w2,_w3,lochierarchy,sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w1,_w2,_w3,sum] InputAdapter - Exchange [s_county,s_state,spark_grouping_id] #2 + Exchange [s_state,s_county,spark_grouping_id] #2 WholeStageCodegen (9) - HashAggregate [s_county,s_state,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [s_county,s_state,ss_net_profit] - Project [s_county,s_state,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -37,7 +37,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (7) @@ -46,23 +46,23 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su InputAdapter Window [_w2,s_state] WholeStageCodegen (6) - Sort [_w2,s_state] + Sort [s_state,_w2] InputAdapter Exchange [s_state] #6 WholeStageCodegen (5) - HashAggregate [s_state,sum] [_w2,s_state,sum,sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_state,sum] [sum(UnscaledValue(ss_net_profit)),s_state,_w2,sum] InputAdapter Exchange [s_state] #7 WholeStageCodegen (4) HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [s_state,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -71,4 +71,4 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 1587213842374..5d01429ad7bf5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_sum] +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] WholeStageCodegen (12) - Project [lochierarchy,rank_within_parent,s_county,s_state,total_sum] + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter - Window [_w1,_w2,_w3] + Window [_w3,_w1,_w2] WholeStageCodegen (11) Sort [_w1,_w2,_w3] InputAdapter Exchange [_w1,_w2] #1 WholeStageCodegen (10) - HashAggregate [s_county,s_state,spark_grouping_id,sum] [_w1,_w2,_w3,lochierarchy,sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w1,_w2,_w3,sum] InputAdapter - Exchange [s_county,s_state,spark_grouping_id] #2 + Exchange [s_state,s_county,spark_grouping_id] #2 WholeStageCodegen (9) - HashAggregate [s_county,s_state,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [s_county,s_state,ss_net_profit] - Project [s_county,s_state,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -37,7 +37,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (7) @@ -46,29 +46,29 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su InputAdapter Window [_w2,s_state] WholeStageCodegen (6) - Sort [_w2,s_state] + Sort [s_state,_w2] InputAdapter Exchange [s_state] #6 WholeStageCodegen (5) - HashAggregate [s_state,sum] [_w2,s_state,sum,sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_state,sum] [sum(UnscaledValue(ss_net_profit)),s_state,_w2,sum] InputAdapter Exchange [s_state] #7 WholeStageCodegen (4) HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [s_state,ss_net_profit] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [s_state,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_sold_date_sk,ss_store_sk] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_net_profit,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/simplified.txt index cca51b9457474..7fb18bbd622ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/simplified.txt @@ -1,65 +1,65 @@ WholeStageCodegen (11) - Sort [brand_id,ext_price] + Sort [ext_price,brand_id] InputAdapter - Exchange [brand_id,ext_price] #1 + Exchange [ext_price,brand_id] #1 WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,sum,t_hour,t_minute] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ext_price))] + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 WholeStageCodegen (9) - HashAggregate [ext_price,i_brand,i_brand_id,t_hour,t_minute] [sum,sum] - Project [ext_price,i_brand,i_brand_id,t_hour,t_minute] - BroadcastHashJoin [t_time_sk,time_sk] - Project [ext_price,i_brand,i_brand_id,time_sk] + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] BroadcastHashJoin [i_item_sk,sold_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter Union WholeStageCodegen (3) Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk,ws_sold_time_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk,ws_sold_time_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (5) Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk,cs_sold_time_sk] + Filter [cs_sold_date_sk,cs_item_sk,cs_sold_time_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk,cs_sold_time_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #4 WholeStageCodegen (7) Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_sold_time_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_sold_time_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - Project [t_hour,t_minute,t_time_sk] + Project [t_time_sk,t_hour,t_minute] Filter [t_meal_time,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_meal_time,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index cca51b9457474..7fb18bbd622ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -1,65 +1,65 @@ WholeStageCodegen (11) - Sort [brand_id,ext_price] + Sort [ext_price,brand_id] InputAdapter - Exchange [brand_id,ext_price] #1 + Exchange [ext_price,brand_id] #1 WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,sum,t_hour,t_minute] [brand,brand_id,ext_price,sum,sum(UnscaledValue(ext_price))] + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 WholeStageCodegen (9) - HashAggregate [ext_price,i_brand,i_brand_id,t_hour,t_minute] [sum,sum] - Project [ext_price,i_brand,i_brand_id,t_hour,t_minute] - BroadcastHashJoin [t_time_sk,time_sk] - Project [ext_price,i_brand,i_brand_id,time_sk] + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] BroadcastHashJoin [i_item_sk,sold_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_brand,i_brand_id,i_item_sk] - Filter [i_item_sk,i_manager_id] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_brand_id,i_item_sk,i_manager_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter Union WholeStageCodegen (3) Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk,ws_sold_time_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk,ws_sold_time_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_sold_time_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (5) Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_date_sk,cs_sold_time_sk] + Filter [cs_sold_date_sk,cs_item_sk,cs_sold_time_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk,cs_sold_time_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_sold_time_sk,cs_item_sk,cs_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #4 WholeStageCodegen (7) Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_sold_time_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_sold_time_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_sold_time_sk,ss_item_sk,ss_ext_sales_price] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #5 WholeStageCodegen (8) - Project [t_hour,t_minute,t_time_sk] + Project [t_time_sk,t_hour,t_minute] Filter [t_meal_time,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_meal_time,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 7662bba3282c2..a100b6659f162 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -160,7 +160,7 @@ Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk Output [2]: [d_date_sk#15, d_date#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -168,7 +168,7 @@ Input [2]: [d_date_sk#15, d_date#16] (20) Filter [codegen id : 3] Input [2]: [d_date_sk#15, d_date#16] -Condition : (isnotnull(d_date_sk#15) AND isnotnull(d_date#16)) +Condition : (isnotnull(d_date#16) AND isnotnull(d_date_sk#15)) (21) BroadcastExchange Input [2]: [d_date_sk#15, d_date#16] @@ -226,7 +226,7 @@ Input [8]: [cs_sold_date_sk#4, cs_item_sk#8, cs_promo_sk#9, cs_order_number#10, Output [2]: [d_date_sk#22, d_week_seq#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 9] @@ -234,7 +234,7 @@ Input [2]: [d_date_sk#22, d_week_seq#23] (35) Filter [codegen id : 9] Input [2]: [d_date_sk#22, d_week_seq#23] -Condition : (isnotnull(d_date_sk#22) AND isnotnull(d_week_seq#23)) +Condition : (isnotnull(d_week_seq#23) AND isnotnull(d_date_sk#22)) (36) Scan parquet default.date_dim Output [4]: [d_date_sk#24, d_date#25, d_week_seq#26, d_year#27] @@ -385,7 +385,7 @@ Arguments: [cs_item_sk#8 ASC NULLS FIRST, cs_order_number#10 ASC NULLS FIRST], f Output [2]: [cr_item_sk#42, cr_order_number#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (70) ColumnarToRow [codegen id : 18] @@ -393,7 +393,7 @@ Input [2]: [cr_item_sk#42, cr_order_number#43] (71) Filter [codegen id : 18] Input [2]: [cr_item_sk#42, cr_order_number#43] -Condition : (isnotnull(cr_order_number#43) AND isnotnull(cr_item_sk#42)) +Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index d0b53caa76851..39dba3af02359 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehouse_name] +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] WholeStageCodegen (21) - HashAggregate [count,d_week_seq,i_item_desc,w_warehouse_name] [count,count(1),no_promo,promo,total_cnt] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter - Exchange [d_week_seq,i_item_desc,w_warehouse_name] #1 + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 WholeStageCodegen (20) - HashAggregate [d_week_seq,i_item_desc,w_warehouse_name] [count,count] - Project [d_week_seq,i_item_desc,w_warehouse_name] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] WholeStageCodegen (17) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #2 WholeStageCodegen (16) - Project [cs_item_sk,cs_order_number,d_week_seq,i_item_desc,w_warehouse_name] + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,d_week_seq,i_item_desc,w_warehouse_name] - SortMergeJoin [cs_item_sk,cs_quantity,d_date_sk,inv_date_sk,inv_item_sk,inv_quantity_on_hand] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter WholeStageCodegen (11) Sort [cs_item_sk,d_date_sk] InputAdapter Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,d_date_sk,d_week_seq,i_item_desc] - BroadcastHashJoin [cs_sold_date_sk,d_date,d_date,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_date_sk,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter WholeStageCodegen (5) @@ -33,12 +33,12 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (4) - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_sold_date_sk,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Project [cs_bill_cdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [hd_demo_sk,cs_bill_hdemo_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -46,16 +46,16 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_marital_status] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] @@ -65,7 +65,7 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (7) Sort [i_item_sk] @@ -75,43 +75,43 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Project [d_date,d_date_sk,d_date_sk,d_week_seq] + Project [d_date_sk,d_date_sk,d_date,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) - Project [d_date,d_date_sk,d_week_seq] - Filter [d_date,d_date_sk,d_week_seq,d_year] + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_week_seq,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter WholeStageCodegen (14) - Sort [inv_date_sk,inv_item_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_date_sk,inv_item_sk] #11 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [w_warehouse_name,inv_date_sk,inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [w_warehouse_sk,inv_warehouse_sk] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #13 WholeStageCodegen (15) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 9477ce20210bf..5ca28c3e30adf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -263,7 +263,7 @@ Input [11]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, Output [2]: [d_date_sk#31, d_week_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 7] @@ -271,7 +271,7 @@ Input [2]: [d_date_sk#31, d_week_seq#32] (45) Filter [codegen id : 7] Input [2]: [d_date_sk#31, d_week_seq#32] -Condition : (isnotnull(d_date_sk#31) AND isnotnull(d_week_seq#32)) +Condition : (isnotnull(d_week_seq#32) AND isnotnull(d_date_sk#31)) (46) BroadcastExchange Input [2]: [d_date_sk#31, d_week_seq#32] @@ -290,7 +290,7 @@ Input [11]: [cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, Output [2]: [d_date_sk#34, d_date#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 8] @@ -298,7 +298,7 @@ Input [2]: [d_date_sk#34, d_date#35] (51) Filter [codegen id : 8] Input [2]: [d_date_sk#34, d_date#35] -Condition : (isnotnull(d_date_sk#34) AND isnotnull(d_date#35)) +Condition : (isnotnull(d_date#35) AND isnotnull(d_date_sk#34)) (52) BroadcastExchange Input [2]: [d_date_sk#34, d_date#35] @@ -344,7 +344,7 @@ Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#15, Output [2]: [cr_item_sk#39, cr_order_number#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (62) ColumnarToRow [codegen id : 10] @@ -352,7 +352,7 @@ Input [2]: [cr_item_sk#39, cr_order_number#40] (63) Filter [codegen id : 10] Input [2]: [cr_item_sk#39, cr_order_number#40] -Condition : (isnotnull(cr_order_number#40) AND isnotnull(cr_item_sk#39)) +Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) (64) BroadcastExchange Input [2]: [cr_item_sk#39, cr_order_number#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 1488d52118b82..25f03cbac8d8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -1,60 +1,60 @@ -TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehouse_name] +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] WholeStageCodegen (12) - HashAggregate [count,d_week_seq,i_item_desc,w_warehouse_name] [count,count(1),no_promo,promo,total_cnt] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter - Exchange [d_week_seq,i_item_desc,w_warehouse_name] #1 + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 WholeStageCodegen (11) - HashAggregate [d_week_seq,i_item_desc,w_warehouse_name] [count,count] - Project [d_week_seq,i_item_desc,w_warehouse_name] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_item_sk,cs_order_number,d_week_seq,i_item_desc,w_warehouse_name] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,d_week_seq,i_item_desc,w_warehouse_name] - BroadcastHashJoin [cs_ship_date_sk,d_date,d_date,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,d_date,d_week_seq,i_item_desc,w_warehouse_name] - BroadcastHashJoin [d_date_sk,d_week_seq,d_week_seq,inv_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,d_date,d_week_seq,i_item_desc,inv_date_sk,w_warehouse_name] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,i_item_desc,inv_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,i_item_desc,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Project [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,i_item_desc,inv_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,inv_date_sk,inv_warehouse_sk] - BroadcastHashJoin [cs_item_sk,cs_quantity,inv_item_sk,inv_quantity_on_hand] - Filter [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,inv_warehouse_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_marital_status] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] @@ -65,19 +65,19 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) - Project [d_date,d_date_sk,d_week_seq] - Filter [d_date,d_date_sk,d_week_seq,d_year] + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_week_seq,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] @@ -87,7 +87,7 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt index 57379ef90be5e..af8527f155c8e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/simplified.txt @@ -3,8 +3,8 @@ WholeStageCodegen (10) InputAdapter Exchange [cnt] #1 WholeStageCodegen (9) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] @@ -12,29 +12,29 @@ WholeStageCodegen (10) Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_customer_sk] #3 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -42,15 +42,15 @@ WholeStageCodegen (10) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] @@ -60,4 +60,4 @@ WholeStageCodegen (10) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 55312b6569a21..46b7241565719 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -3,32 +3,32 @@ WholeStageCodegen (7) InputAdapter Exchange [cnt] #1 WholeStageCodegen (6) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk] #2 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,19 +36,19 @@ WholeStageCodegen (7) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt index c35e70d72eb36..c8cfa693e88ba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/simplified.txt @@ -1,10 +1,10 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (35) - Project [customer_first_name,customer_id,customer_last_name] + Project [customer_id,customer_first_name,customer_last_name] SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_first_name,customer_id,customer_id,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) @@ -16,29 +16,29 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Exchange [customer_id] #1 WholeStageCodegen (7) Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #2 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #3 WholeStageCodegen (2) - Project [d_year,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -48,39 +48,39 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (4) - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter WholeStageCodegen (16) Sort [customer_id] InputAdapter Exchange [customer_id] #6 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_first_name,customer_id,customer_last_name,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #7 WholeStageCodegen (14) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (10) - Project [d_year,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -88,7 +88,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (13) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_id,c_customer_sk,c_first_name,c_last_name] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 InputAdapter WholeStageCodegen (25) Sort [customer_id] @@ -97,61 +97,61 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (24) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 WholeStageCodegen (23) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (20) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #12 WholeStageCodegen (19) - Project [d_year,ws_bill_customer_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] InputAdapter ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_id,c_customer_sk,c_first_name,c_last_name] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 WholeStageCodegen (32) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (29) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 WholeStageCodegen (28) - Project [d_year,ws_bill_customer_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] InputAdapter ReusedExchange [d_date_sk,d_year] #9 InputAdapter WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_id,c_customer_sk,c_first_name,c_last_name] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 68a6e7bfd91a3..add2d43fc5807 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -1,60 +1,60 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] WholeStageCodegen (16) - Project [customer_first_name,customer_id,customer_last_name] + Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_first_name,customer_id,customer_id,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_sold_date_sk,ss_net_paid] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_first_name,customer_id,customer_last_name,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_sold_date_sk,ss_net_paid] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_net_paid] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -63,45 +63,45 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (11) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #8 WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_sold_date_sk,ws_net_paid] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #10 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 WholeStageCodegen (14) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_sold_date_sk,ws_net_paid] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #9 + ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index f5129e36b4a23..39748bdd2772b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -157,7 +157,7 @@ Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_manufact_id), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -165,7 +165,7 @@ Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category (6) Filter [codegen id : 1] Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] -Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_manufact_id#11)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] @@ -397,7 +397,7 @@ Arguments: [cast(ws_order_number#40 as bigint) ASC NULLS FIRST, cast(ws_item_sk# Output [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 21] @@ -405,7 +405,7 @@ Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_ (61) Filter [codegen id : 21] Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] -Condition : (isnotnull(wr_item_sk#44) AND isnotnull(wr_order_number#45)) +Condition : (isnotnull(wr_order_number#45) AND isnotnull(wr_item_sk#44)) (62) Exchange Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/simplified.txt index 40aa2931ad5b8..d8d1a3976559d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/simplified.txt @@ -1,237 +1,237 @@ -TakeOrderedAndProject [curr_yr_cnt,i_brand_id,i_category_id,i_class_id,i_manufact_id,prev_year,prev_yr_cnt,sales_amt_diff,sales_cnt_diff,year] +TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] WholeStageCodegen (55) - Project [d_year,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_amt,sales_cnt,sales_cnt] - SortMergeJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,i_manufact_id,i_manufact_id,sales_cnt,sales_cnt] + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter WholeStageCodegen (27) - Sort [i_brand_id,i_category_id,i_class_id,i_manufact_id] + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id,i_manufact_id] #1 + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 WholeStageCodegen (26) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #2 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 WholeStageCodegen (25) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #3 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 WholeStageCodegen (24) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (16) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #4 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #4 WholeStageCodegen (15) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (7) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #5 + Exchange [cs_order_number,cs_item_sk] #5 WholeStageCodegen (3) - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Project [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] - Filter [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 + Exchange [cr_order_number,cr_item_sk] #8 WholeStageCodegen (5) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] WholeStageCodegen (14) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] + 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] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] WholeStageCodegen (11) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #9 + Exchange [ss_ticket_number,ss_item_sk] #9 WholeStageCodegen (10) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #7 WholeStageCodegen (13) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #10 + Exchange [sr_ticket_number,sr_item_sk] #10 WholeStageCodegen (12) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] WholeStageCodegen (23) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] + 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] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] WholeStageCodegen (20) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #11 + Exchange [ws_order_number,ws_item_sk] #11 WholeStageCodegen (19) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #7 WholeStageCodegen (22) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_item_sk,wr_order_number] #12 + Exchange [wr_order_number,wr_item_sk] #12 WholeStageCodegen (21) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] InputAdapter WholeStageCodegen (54) - Sort [i_brand_id,i_category_id,i_class_id,i_manufact_id] + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id,i_manufact_id] #13 + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 WholeStageCodegen (53) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #14 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 WholeStageCodegen (52) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #15 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 WholeStageCodegen (51) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (43) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #16 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #16 WholeStageCodegen (42) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (34) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (31) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #17 + Exchange [cs_order_number,cs_item_sk] #17 WholeStageCodegen (30) - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter BroadcastExchange #18 WholeStageCodegen (29) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (33) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] #8 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 WholeStageCodegen (41) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] + 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] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] WholeStageCodegen (38) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #19 + Exchange [ss_ticket_number,ss_item_sk] #19 WholeStageCodegen (37) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #18 WholeStageCodegen (40) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] #10 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 WholeStageCodegen (50) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] + 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] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] WholeStageCodegen (47) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #20 + Exchange [ws_order_number,ws_item_sk] #20 WholeStageCodegen (46) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #18 WholeStageCodegen (49) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] #12 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index afdfb4f691a7b..292a44930ed3d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -136,7 +136,7 @@ Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_manufact_id), IsNotNull(i_category_id), IsNotNull(i_brand_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -144,7 +144,7 @@ Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category (6) Filter [codegen id : 1] Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] -Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_manufact_id#11)) AND isnotnull(i_category_id#9)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 4974c17705d87..298a06b87762f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -1,180 +1,180 @@ -TakeOrderedAndProject [curr_yr_cnt,i_brand_id,i_category_id,i_class_id,i_manufact_id,prev_year,prev_yr_cnt,sales_amt_diff,sales_cnt_diff,year] +TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] WholeStageCodegen (34) - Project [d_year,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_amt,sales_cnt,sales_cnt] - BroadcastHashJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,i_manufact_id,i_manufact_id,sales_cnt,sales_cnt] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #1 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 WholeStageCodegen (16) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #2 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #2 WholeStageCodegen (15) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (10) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #3 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 WholeStageCodegen (9) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (4) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Project [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] - Filter [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] WholeStageCodegen (8) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + 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] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] WholeStageCodegen (14) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + 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] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #8 WholeStageCodegen (13) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] InputAdapter BroadcastExchange #9 WholeStageCodegen (33) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #10 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #10 WholeStageCodegen (32) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #11 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #11 WholeStageCodegen (31) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (26) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #12 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #12 WholeStageCodegen (25) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (20) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter BroadcastExchange #13 WholeStageCodegen (18) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] #6 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #6 WholeStageCodegen (24) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + 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] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #13 InputAdapter - ReusedExchange [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] #7 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #7 WholeStageCodegen (30) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + 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] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #13 InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] #8 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt index 74214da0678c6..2da0d94f91e7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/simplified.txt @@ -1,68 +1,68 @@ -TakeOrderedAndProject [channel,col_name,d_qoy,d_year,i_category,sales_amt,sales_cnt] +TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] WholeStageCodegen (11) - HashAggregate [channel,col_name,count,d_qoy,d_year,i_category,sum] [count,count(1),sales_amt,sales_cnt,sum,sum(UnscaledValue(ext_sales_price))] + HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] InputAdapter - Exchange [channel,col_name,d_qoy,d_year,i_category] #1 + Exchange [channel,col_name,d_year,d_qoy,i_category] #1 WholeStageCodegen (10) - HashAggregate [channel,col_name,d_qoy,d_year,ext_sales_price,i_category] [count,count,sum,sum] + HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] InputAdapter Union WholeStageCodegen (3) - Project [d_qoy,d_year,i_category,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_qoy,d_year,ss_ext_sales_price,ss_item_sk,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_category] WholeStageCodegen (6) - Project [d_qoy,d_year,i_category,ws_ext_sales_price,ws_ship_customer_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] + BroadcastHashJoin [ws_item_sk,i_item_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (5) - Project [d_qoy,d_year,ws_ext_sales_price,ws_item_sk,ws_ship_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [ws_item_sk,ws_ship_customer_sk,ws_sold_date_sk] + Filter [ws_ship_customer_sk,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_ship_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price] Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_category] WholeStageCodegen (9) - Project [cs_ext_sales_price,cs_ship_addr_sk,d_qoy,d_year,i_category] + Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_ship_addr_sk,d_qoy,d_year] + Project [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,d_year,d_qoy] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_ship_addr_sk,cs_sold_date_sk] + Filter [cs_ship_addr_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_ship_addr_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #2 InputAdapter - ReusedExchange [i_category,i_item_sk] #3 + ReusedExchange [i_item_sk,i_category] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 697757ade4997..f01916baaac78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -1,58 +1,58 @@ -TakeOrderedAndProject [channel,col_name,d_qoy,d_year,i_category,sales_amt,sales_cnt] +TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] WholeStageCodegen (11) - HashAggregate [channel,col_name,count,d_qoy,d_year,i_category,sum] [count,count(1),sales_amt,sales_cnt,sum,sum(UnscaledValue(ext_sales_price))] + HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] InputAdapter - Exchange [channel,col_name,d_qoy,d_year,i_category] #1 + Exchange [channel,col_name,d_year,d_qoy,i_category] #1 WholeStageCodegen (10) - HashAggregate [channel,col_name,d_qoy,d_year,ext_sales_price,i_category] [count,count,sum,sum] + HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] InputAdapter Union WholeStageCodegen (3) - Project [d_qoy,d_year,i_category,ss_ext_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_category,ss_ext_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_store_sk,ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] WholeStageCodegen (6) - Project [d_qoy,d_year,i_category,ws_ext_sales_price,ws_ship_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_category,ws_ext_sales_price,ws_ship_customer_sk,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Filter [ws_item_sk,ws_ship_customer_sk,ws_sold_date_sk] + Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ship_customer_sk,ws_ext_sales_price,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_ship_customer_sk,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_ship_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price] InputAdapter - ReusedExchange [i_category,i_item_sk] #2 + ReusedExchange [i_item_sk,i_category] #2 InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 WholeStageCodegen (9) - Project [cs_ext_sales_price,cs_ship_addr_sk,d_qoy,d_year,i_category] + Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_ship_addr_sk,cs_sold_date_sk,i_category] + Project [cs_sold_date_sk,cs_ship_addr_sk,cs_ext_sales_price,i_category] BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_ship_addr_sk,cs_sold_date_sk] + Filter [cs_ship_addr_sk,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_ship_addr_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price] InputAdapter - ReusedExchange [i_category,i_item_sk] #2 + ReusedExchange [i_item_sk,i_category] #2 InputAdapter - ReusedExchange [d_date_sk,d_qoy,d_year] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/simplified.txt index cbddf82f08155..6840f4367260f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/simplified.txt @@ -1,29 +1,29 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (25) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,spark_grouping_id,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 WholeStageCodegen (24) - HashAggregate [channel,id,profit,returns,sales,spark_grouping_id] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Expand [channel,id,profit,returns,sales] + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] InputAdapter Union WholeStageCodegen (8) - Project [profit,profit_loss,returns,s_store_sk,sales] + Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #2 WholeStageCodegen (3) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [s_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -42,29 +42,29 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #5 WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(sr_return_amt))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [s_store_sk] #6 WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_net_loss,sr_return_amt] [sum,sum,sum,sum] - Project [s_store_sk,sr_net_loss,sr_return_amt] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [s_store_sk,sr_net_loss,sr_return_amt,sr_returned_date_sk] - BroadcastHashJoin [s_store_sk,sr_store_sk] + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [sr_returned_date_sk,sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter ReusedExchange [s_store_sk] #4 InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (15) - Project [cs_call_center_sk,profit,profit_loss,returns,sales] + Project [sales,returns,profit,profit_loss,cs_call_center_sk] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [cs_call_center_sk] #7 WholeStageCodegen (10) @@ -74,40 +74,40 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_ext_sales_price,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #3 BroadcastExchange #8 WholeStageCodegen (14) - HashAggregate [sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(cr_net_loss)),sum(UnscaledValue(cr_return_amount))] + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange #9 WholeStageCodegen (13) - HashAggregate [cr_net_loss,cr_return_amount] [sum,sum,sum,sum] - Project [cr_net_loss,cr_return_amount] + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (23) - Project [profit,profit_loss,returns,sales,wp_web_page_sk] + Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [sum,sum,wp_web_page_sk] [profit,sales,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [wp_web_page_sk] #10 WholeStageCodegen (18) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] - Project [ws_ext_sales_price,ws_net_profit,ws_web_page_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_page_sk,ws_ext_sales_price,ws_net_profit] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -120,19 +120,19 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #12 WholeStageCodegen (22) - HashAggregate [sum,sum,wp_web_page_sk] [profit_loss,returns,sum,sum,sum(UnscaledValue(wr_net_loss)),sum(UnscaledValue(wr_return_amt))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [wp_web_page_sk] #13 WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,wr_net_loss,wr_return_amt] [sum,sum,sum,sum] - Project [wp_web_page_sk,wr_net_loss,wr_return_amt] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] - Project [wp_web_page_sk,wr_net_loss,wr_return_amt,wr_returned_date_sk] - BroadcastHashJoin [wp_web_page_sk,wr_web_page_sk] + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Project [wr_returned_date_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] Filter [wr_returned_date_sk,wr_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_net_loss,wr_return_amt,wr_returned_date_sk,wr_web_page_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_web_page_sk,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [wp_web_page_sk] #11 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index 591c1e7bc2720..bfbeff02b63be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -1,29 +1,29 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (25) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,spark_grouping_id,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 WholeStageCodegen (24) - HashAggregate [channel,id,profit,returns,sales,spark_grouping_id] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Expand [channel,id,profit,returns,sales] + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] InputAdapter Union WholeStageCodegen (8) - Project [profit,profit_loss,returns,s_store_sk,sales] + Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #2 WholeStageCodegen (3) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [s_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -42,30 +42,30 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #5 WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(sr_return_amt))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [s_store_sk] #6 WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_net_loss,sr_return_amt] [sum,sum,sum,sum] - Project [s_store_sk,sr_net_loss,sr_return_amt] - BroadcastHashJoin [s_store_sk,sr_store_sk] - Project [sr_net_loss,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [s_store_sk] #4 WholeStageCodegen (15) - Project [cs_call_center_sk,profit,profit_loss,returns,sales] + Project [sales,returns,profit,profit_loss,cs_call_center_sk] InputAdapter BroadcastNestedLoopJoin BroadcastExchange #7 WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [cs_call_center_sk] #8 WholeStageCodegen (10) @@ -75,39 +75,39 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_ext_sales_price,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (14) - HashAggregate [sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(cr_net_loss)),sum(UnscaledValue(cr_return_amount))] + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange #9 WholeStageCodegen (13) - HashAggregate [cr_net_loss,cr_return_amount] [sum,sum,sum,sum] - Project [cr_net_loss,cr_return_amount] + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (23) - Project [profit,profit_loss,returns,sales,wp_web_page_sk] + Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [sum,sum,wp_web_page_sk] [profit,sales,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [wp_web_page_sk] #10 WholeStageCodegen (18) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] - Project [ws_ext_sales_price,ws_net_profit,ws_web_page_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_page_sk,ws_ext_sales_price,ws_net_profit] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -120,19 +120,19 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #12 WholeStageCodegen (22) - HashAggregate [sum,sum,wp_web_page_sk] [profit_loss,returns,sum,sum,sum(UnscaledValue(wr_net_loss)),sum(UnscaledValue(wr_return_amt))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [wp_web_page_sk] #13 WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,wr_net_loss,wr_return_amt] [sum,sum,sum,sum] - Project [wp_web_page_sk,wr_net_loss,wr_return_amt] - BroadcastHashJoin [wp_web_page_sk,wr_web_page_sk] - Project [wr_net_loss,wr_return_amt,wr_web_page_sk] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_returned_date_sk,wr_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_net_loss,wr_return_amt,wr_returned_date_sk,wr_web_page_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_web_page_sk,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt index 633284b9d2200..05cf4c2490b7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt @@ -285,7 +285,7 @@ Input [12]: [ss_sold_year#25, ss_item_sk#2, ss_customer_sk#3, ss_qty#26, ss_wc#2 Output [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_number#58, ws_quantity#59, ws_wholesale_cost#60, ws_sales_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 16] @@ -293,7 +293,7 @@ Input [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_ (49) Filter [codegen id : 16] Input [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_number#58, ws_quantity#59, ws_wholesale_cost#60, ws_sales_price#61] -Condition : ((isnotnull(ws_sold_date_sk#55) AND isnotnull(ws_bill_customer_sk#57)) AND isnotnull(ws_item_sk#56)) +Condition : ((isnotnull(ws_sold_date_sk#55) AND isnotnull(ws_item_sk#56)) AND isnotnull(ws_bill_customer_sk#57)) (50) Exchange Input [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_number#58, ws_quantity#59, ws_wholesale_cost#60, ws_sales_price#61] @@ -382,10 +382,10 @@ Right keys [3]: [ws_sold_year#76, ws_item_sk#56, ws_customer_sk#77] Join condition: None (69) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#26 as double) / cast(coalesce((ws_qty#78 + cs_qty#52), 1) as double)), 2) AS ratio#81, ss_qty#26 AS store_qty#82, ss_wc#27 AS store_wholesale_cost#83, ss_sp#28 AS store_sales_price#84, (coalesce(ws_qty#78, 0) + coalesce(cs_qty#52, 0)) AS other_chan_qty#85, CheckOverflow((promote_precision(cast(coalesce(ws_wc#79, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#53, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#86, CheckOverflow((promote_precision(cast(coalesce(ws_sp#80, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#54, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#87, ss_sp#28, ss_qty#26, ws_qty#78, ss_wc#27, cs_qty#52] +Output [12]: [round((cast(ss_qty#26 as double) / cast(coalesce((ws_qty#78 + cs_qty#52), 1) as double)), 2) AS ratio#81, ss_qty#26 AS store_qty#82, ss_wc#27 AS store_wholesale_cost#83, ss_sp#28 AS store_sales_price#84, (coalesce(ws_qty#78, 0) + coalesce(cs_qty#52, 0)) AS other_chan_qty#85, CheckOverflow((promote_precision(cast(coalesce(ws_wc#79, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#53, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#86, CheckOverflow((promote_precision(cast(coalesce(ws_sp#80, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#54, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#87, ss_qty#26, ss_wc#27, ss_sp#28, ws_qty#78, cs_qty#52] Input [15]: [ss_sold_year#25, ss_item_sk#2, ss_customer_sk#3, ss_qty#26, ss_wc#27, ss_sp#28, cs_qty#52, cs_wc#53, cs_sp#54, ws_sold_year#76, ws_item_sk#56, ws_customer_sk#77, ws_qty#78, ws_wc#79, ws_sp#80] (70) TakeOrderedAndProject -Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sp#28, ss_qty#26, ws_qty#78, ss_wc#27, cs_qty#52] +Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#26, ss_wc#27, ss_sp#28, ws_qty#78, cs_qty#52] Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#26 DESC NULLS LAST, ss_wc#27 DESC NULLS LAST, ss_sp#28 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#26 as double) / cast(coalesce((ws_qty#78 + cs_qty#52), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/simplified.txt index 726d382498db7..b46178cb8f473 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/simplified.txt @@ -1,80 +1,80 @@ -TakeOrderedAndProject [cs_qty,other_chan_qty,other_chan_sales_price,other_chan_wholesale_cost,ratio,ss_qty,ss_sp,ss_wc,store_qty,store_sales_price,store_wholesale_cost,ws_qty] +TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (23) - Project [cs_qty,cs_sp,cs_wc,ss_qty,ss_sp,ss_wc,ws_qty,ws_sp,ws_wc] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_sold_year,ws_customer_sk,ws_item_sk,ws_sold_year] + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter WholeStageCodegen (15) - Project [cs_qty,cs_sp,cs_wc,ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc] - SortMergeJoin [cs_customer_sk,cs_item_sk,cs_sold_year,ss_customer_sk,ss_item_sk,ss_sold_year] + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,cs_qty,cs_wc,cs_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk,ss_item_sk,ss_sold_year] - HashAggregate [d_year,ss_customer_sk,ss_item_sk,sum,sum,sum] [ss_qty,ss_sold_year,ss_sp,ss_wc,sum,sum,sum,sum(UnscaledValue(ss_sales_price)),sum(UnscaledValue(ss_wholesale_cost)),sum(cast(ss_quantity as bigint))] + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ss_customer_sk,ss_item_sk] #1 + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 WholeStageCodegen (6) - HashAggregate [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_wholesale_cost] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] Filter [sr_ticket_number] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_item_sk] #2 WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price] WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #3 + Exchange [sr_ticket_number,sr_item_sk] #3 WholeStageCodegen (3) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (14) - Sort [cs_customer_sk,cs_item_sk,cs_sold_year] + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] - HashAggregate [cs_bill_customer_sk,cs_item_sk,d_year,sum,sum,sum] [cs_customer_sk,cs_qty,cs_sold_year,cs_sp,cs_wc,sum,sum,sum,sum(UnscaledValue(cs_sales_price)),sum(UnscaledValue(cs_wholesale_cost)),sum(cast(cs_quantity as bigint))] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cast(cs_quantity as bigint)),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk,d_year] #5 + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #5 WholeStageCodegen (13) - HashAggregate [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] Filter [cr_order_number] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (9) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #6 + Exchange [cs_order_number,cs_item_sk] #6 WholeStageCodegen (8) - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_item_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price] WholeStageCodegen (11) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #7 + Exchange [cr_order_number,cr_item_sk] #7 WholeStageCodegen (10) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_order_number] @@ -82,34 +82,34 @@ TakeOrderedAndProject [cs_qty,other_chan_qty,other_chan_sales_price,other_chan_w ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) - Sort [ws_customer_sk,ws_item_sk,ws_sold_year] + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] - HashAggregate [d_year,sum,sum,sum,ws_bill_customer_sk,ws_item_sk] [sum,sum,sum,sum(UnscaledValue(ws_sales_price)),sum(UnscaledValue(ws_wholesale_cost)),sum(cast(ws_quantity as bigint)),ws_customer_sk,ws_qty,ws_sold_year,ws_sp,ws_wc] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(cast(ws_quantity as bigint)),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ws_bill_customer_sk,ws_item_sk] #8 + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #8 WholeStageCodegen (21) - HashAggregate [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] Filter [wr_order_number] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] WholeStageCodegen (17) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #9 + Exchange [ws_order_number,ws_item_sk] #9 WholeStageCodegen (16) - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Filter [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price] WholeStageCodegen (19) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_item_sk,wr_order_number] #10 + Exchange [wr_order_number,wr_item_sk] #10 WholeStageCodegen (18) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index a881456ea09f8..1bcf039ddfaf1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -332,10 +332,10 @@ Right keys [3]: [cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75] Join condition: None (59) Project [codegen id : 12] -Output [12]: [round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) AS ratio#80, ss_qty#25 AS store_qty#81, ss_wc#26 AS store_wholesale_cost#82, ss_sp#27 AS store_sales_price#83, (coalesce(ws_qty#50, 0) + coalesce(cs_qty#76, 0)) AS other_chan_qty#84, CheckOverflow((promote_precision(cast(coalesce(ws_wc#51, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#77, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#85, CheckOverflow((promote_precision(cast(coalesce(ws_sp#52, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#78, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#86, ss_sp#27, ss_wc#26, cs_qty#76, ss_qty#25, ws_qty#50] +Output [12]: [round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) AS ratio#80, ss_qty#25 AS store_qty#81, ss_wc#26 AS store_wholesale_cost#82, ss_sp#27 AS store_sales_price#83, (coalesce(ws_qty#50, 0) + coalesce(cs_qty#76, 0)) AS other_chan_qty#84, CheckOverflow((promote_precision(cast(coalesce(ws_wc#51, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#77, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#85, CheckOverflow((promote_precision(cast(coalesce(ws_sp#52, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#78, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#86, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, cs_qty#76] Input [15]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, ws_wc#51, ws_sp#52, cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75, cs_qty#76, cs_wc#77, cs_sp#78] (60) TakeOrderedAndProject -Input [12]: [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86, ss_sp#27, ss_wc#26, cs_qty#76, ss_qty#25, ws_qty#50] +Input [12]: [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, cs_qty#76] Arguments: 100, [ratio#80 ASC NULLS FIRST, ss_qty#25 DESC NULLS LAST, ss_wc#26 DESC NULLS LAST, ss_sp#27 DESC NULLS LAST, other_chan_qty#84 ASC NULLS FIRST, other_chan_wholesale_cost#85 ASC NULLS FIRST, other_chan_sales_price#86 ASC NULLS FIRST, round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) ASC NULLS FIRST], [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt index ffff01466ef21..9770702831acc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [cs_qty,other_chan_qty,other_chan_sales_price,other_chan_wholesale_cost,ratio,ss_qty,ss_sp,ss_wc,store_qty,store_sales_price,store_wholesale_cost,ws_qty] +TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (12) - Project [cs_qty,cs_sp,cs_wc,ss_qty,ss_sp,ss_wc,ws_qty,ws_sp,ws_wc] - BroadcastHashJoin [cs_customer_sk,cs_item_sk,cs_sold_year,ss_customer_sk,ss_item_sk,ss_sold_year] - Project [ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc,ws_qty,ws_sp,ws_wc] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_sold_year,ws_customer_sk,ws_item_sk,ws_sold_year] - HashAggregate [d_year,ss_customer_sk,ss_item_sk,sum,sum,sum] [ss_qty,ss_sold_year,ss_sp,ss_wc,sum,sum,sum,sum(UnscaledValue(ss_sales_price)),sum(UnscaledValue(ss_wholesale_cost)),sum(cast(ss_quantity as bigint))] + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] + BroadcastHashJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + BroadcastHashJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ss_customer_sk,ss_item_sk] #1 + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 WholeStageCodegen (3) - HashAggregate [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_wholesale_cost] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] Filter [sr_ticket_number] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -36,24 +36,24 @@ TakeOrderedAndProject [cs_qty,other_chan_qty,other_chan_sales_price,other_chan_w BroadcastExchange #4 WholeStageCodegen (7) Filter [ws_qty] - HashAggregate [d_year,sum,sum,sum,ws_bill_customer_sk,ws_item_sk] [sum,sum,sum,sum(UnscaledValue(ws_sales_price)),sum(UnscaledValue(ws_wholesale_cost)),sum(cast(ws_quantity as bigint)),ws_customer_sk,ws_qty,ws_sold_year,ws_sp,ws_wc] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(cast(ws_quantity as bigint)),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ws_bill_customer_sk,ws_item_sk] #5 + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 WholeStageCodegen (6) - HashAggregate [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] Filter [wr_order_number] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Filter [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_order_number] @@ -63,24 +63,24 @@ TakeOrderedAndProject [cs_qty,other_chan_qty,other_chan_sales_price,other_chan_w BroadcastExchange #7 WholeStageCodegen (11) Filter [cs_qty] - HashAggregate [cs_bill_customer_sk,cs_item_sk,d_year,sum,sum,sum] [cs_customer_sk,cs_qty,cs_sold_year,cs_sp,cs_wc,sum,sum,sum,sum(UnscaledValue(cs_sales_price)),sum(UnscaledValue(cs_wholesale_cost)),sum(cast(cs_quantity as bigint))] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cast(cs_quantity as bigint)),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk,d_year] #8 + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 WholeStageCodegen (10) - HashAggregate [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] Filter [cr_order_number] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Filter [cs_sold_date_sk,cs_item_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/simplified.txt index 8ef698c9f896c..e031b8c4c836c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/simplified.txt @@ -1,52 +1,52 @@ -TakeOrderedAndProject [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number,substr(s_city, 1, 30)] +TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] WholeStageCodegen (9) - Project [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #1 WholeStageCodegen (5) - HashAggregate [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] InputAdapter - Exchange [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #2 WholeStageCodegen (4) - HashAggregate [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Project [s_city,s_store_sk] + Project [s_store_sk,s_city] Filter [s_number_employees,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_number_employees,s_store_sk] + Scan parquet default.store [s_store_sk,s_number_employees,s_city] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index 53f5a7d0acc0e..5b6177488766e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -1,43 +1,43 @@ -TakeOrderedAndProject [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number,substr(s_city, 1, 30)] +TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] WholeStageCodegen (6) - Project [amt,c_first_name,c_last_name,profit,s_city,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - HashAggregate [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number,sum,sum] [amt,profit,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] InputAdapter - Exchange [s_city,ss_addr_sk,ss_customer_sk,ss_ticket_number] #1 + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 WholeStageCodegen (4) - HashAggregate [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] [sum,sum,sum,sum] - Project [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [s_city,ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dow,d_year] + Filter [d_dow,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dow,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dow] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [s_city,s_store_sk] + Project [s_store_sk,s_city] Filter [s_number_employees,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_city,s_number_employees,s_store_sk] + Scan parquet default.store [s_store_sk,s_number_employees,s_city] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt index 415bf96afe55d..b437f17696037 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (15) - HashAggregate [s_store_name,sum] [sum,sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit)] + HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] InputAdapter Exchange [s_store_name] #1 WholeStageCodegen (14) HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [s_store_name,ss_net_profit] - SortMergeJoin [ca_zip,s_zip] + Project [ss_net_profit,s_store_name] + SortMergeJoin [s_zip,ca_zip] InputAdapter WholeStageCodegen (4) Sort [s_zip] InputAdapter Exchange [s_zip] #2 WholeStageCodegen (3) - Project [s_store_name,s_zip,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_net_profit,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Filter [s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (13) Sort [ca_zip] @@ -58,13 +58,13 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (10) Project [ca_zip] Filter [count(1)] - HashAggregate [ca_zip,count] [ca_zip,count,count(1),count(1)] + HashAggregate [ca_zip,count] [count(1),ca_zip,count(1),count] InputAdapter Exchange [ca_zip] #8 WholeStageCodegen (9) HashAggregate [ca_zip] [count,count] Project [ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] + SortMergeJoin [ca_address_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (6) Sort [ca_address_sk] @@ -82,7 +82,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Exchange [c_current_addr_sk] #10 WholeStageCodegen (7) Project [c_current_addr_sk] - Filter [c_current_addr_sk,c_preferred_cust_flag] + Filter [c_preferred_cust_flag,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index b463f20da9968..cc62907dc72e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -1,35 +1,35 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (9) - HashAggregate [s_store_name,sum] [sum,sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit)] + HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] InputAdapter Exchange [s_store_name] #1 WholeStageCodegen (8) HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [s_store_name,ss_net_profit] - BroadcastHashJoin [ca_zip,s_zip] - Project [s_store_name,s_zip,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_net_profit,s_store_name] + BroadcastHashJoin [s_zip,ca_zip] + Project [ss_net_profit,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_qoy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) @@ -49,13 +49,13 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (5) Project [ca_zip] Filter [count(1)] - HashAggregate [ca_zip,count] [ca_zip,count,count(1),count(1)] + HashAggregate [ca_zip,count] [count(1),ca_zip,count(1),count] InputAdapter Exchange [ca_zip] #7 WholeStageCodegen (4) HashAggregate [ca_zip] [count,count] Project [ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] Filter [ca_address_sk] ColumnarToRow InputAdapter @@ -64,7 +64,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] BroadcastExchange #8 WholeStageCodegen (3) Project [c_current_addr_sk] - Filter [c_current_addr_sk,c_preferred_cust_flag] + Filter [c_preferred_cust_flag,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt index a2189f6763eb3..7b73e4307dcf0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt @@ -1,39 +1,39 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (32) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,spark_grouping_id,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 WholeStageCodegen (31) - HashAggregate [channel,id,profit,returns,sales,spark_grouping_id] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Expand [channel,id,profit,returns,sales] + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] InputAdapter Union WholeStageCodegen (10) - HashAggregate [isEmpty,isEmpty,s_store_id,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [s_store_id] #2 WholeStageCodegen (9) - HashAggregate [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_promo_sk,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] WholeStageCodegen (2) Sort [ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_item_sk,ss_ticket_number] #3 WholeStageCodegen (1) - Filter [ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit] WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] InputAdapter @@ -42,7 +42,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [sr_item_sk,sr_ticket_number] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_net_loss,sr_return_amt,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) @@ -50,7 +50,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (6) @@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) @@ -66,40 +66,40 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_tv,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00))] + 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(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cp_catalog_page_id] #9 WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cp_catalog_page_sk,cs_catalog_page_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_promo_sk] + Project [cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] WholeStageCodegen (12) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #10 WholeStageCodegen (11) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_order_number,cs_promo_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit] WholeStageCodegen (14) Sort [cr_item_sk,cr_order_number] InputAdapter @@ -108,7 +108,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cr_item_sk,cr_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_net_loss,cr_order_number,cr_return_amount] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter @@ -121,33 +121,33 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (30) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,web_site_id] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [web_site_id] #13 WholeStageCodegen (29) - HashAggregate [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_web_site_sk] - BroadcastHashJoin [p_promo_sk,ws_promo_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_promo_sk,ws_web_site_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] WholeStageCodegen (22) Sort [ws_item_sk,ws_order_number] InputAdapter Exchange [ws_item_sk,ws_order_number] #14 WholeStageCodegen (21) - Filter [ws_item_sk,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + Filter [ws_sold_date_sk,ws_web_site_sk,ws_item_sk,ws_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_order_number,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit] WholeStageCodegen (24) Sort [wr_item_sk,wr_order_number] InputAdapter @@ -156,7 +156,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [wr_item_sk,wr_order_number] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [i_item_sk] #5 InputAdapter @@ -169,4 +169,4 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 2f2f3b8cd3d25..37f46f14e467b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -1,40 +1,40 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (23) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,spark_grouping_id,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 WholeStageCodegen (22) - HashAggregate [channel,id,profit,returns,sales,spark_grouping_id] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Expand [channel,id,profit,returns,sales] + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] InputAdapter Union WholeStageCodegen (7) - HashAggregate [isEmpty,isEmpty,s_store_id,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [s_store_id] #2 WholeStageCodegen (6) - HashAggregate [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_promo_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [sr_item_sk,sr_ticket_number] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_net_loss,sr_return_amt,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) @@ -65,34 +65,34 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_tv,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (14) - HashAggregate [cp_catalog_page_id,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00))] + 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(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cp_catalog_page_id] #8 WholeStageCodegen (13) - HashAggregate [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit,cs_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk] - BroadcastHashJoin [cp_catalog_page_sk,cs_catalog_page_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk,cs_sold_date_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + Filter [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_order_number,cs_promo_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) Filter [cr_item_sk,cr_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_net_loss,cr_order_number,cr_return_amount] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter @@ -101,38 +101,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #6 InputAdapter ReusedExchange [p_promo_sk] #7 WholeStageCodegen (21) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,web_site_id] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [web_site_id] #11 WholeStageCodegen (20) - HashAggregate [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [p_promo_sk,ws_promo_sk] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_promo_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk,ws_web_site_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [ws_item_sk,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + Filter [ws_sold_date_sk,ws_web_site_sk,ws_item_sk,ws_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_order_number,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit] InputAdapter BroadcastExchange #12 WholeStageCodegen (15) Filter [wr_item_sk,wr_order_number] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter @@ -141,7 +141,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index d76e1217ae004..c603ab5194286 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip,ctr_total_return] +TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] WholeStageCodegen (20) - Project [c_customer_id,c_first_name,c_last_name,c_salutation,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip,ctr_total_return] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,ctr_state,ctr_total_return] - Project [c_customer_id,c_first_name,c_last_name,c_salutation,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip,ctr_state,ctr_total_return] + Project [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + Project [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_state,ctr_total_return] SortMergeJoin [c_customer_sk,ctr_customer_sk] InputAdapter WholeStageCodegen (3) @@ -10,19 +10,19 @@ TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_ci InputAdapter Exchange [c_customer_sk] #1 WholeStageCodegen (2) - Project [c_customer_id,c_customer_sk,c_first_name,c_last_name,c_salutation,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip] + Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_id,c_customer_sk,c_first_name,c_last_name,c_salutation] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip] + Scan parquet 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] InputAdapter WholeStageCodegen (11) Sort [ctr_customer_sk] @@ -30,30 +30,30 @@ TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_ci Exchange [ctr_customer_sk] #3 WholeStageCodegen (10) Filter [ctr_total_return] - HashAggregate [ca_state,cr_returning_customer_sk,sum] [ctr_customer_sk,ctr_state,ctr_total_return,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,cr_returning_customer_sk] #4 + Exchange [cr_returning_customer_sk,ca_state] #4 WholeStageCodegen (9) - HashAggregate [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] [sum,sum] - Project [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] - SortMergeJoin [ca_address_sk,cr_returning_addr_sk] + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + SortMergeJoin [cr_returning_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (6) Sort [cr_returning_addr_sk] InputAdapter Exchange [cr_returning_addr_sk] #5 WholeStageCodegen (5) - Project [cr_return_amt_inc_tax,cr_returning_addr_sk,cr_returning_customer_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_return_amt_inc_tax,cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -71,30 +71,30 @@ TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_ci BroadcastExchange #8 WholeStageCodegen (19) Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [count,ctr_state,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_state,sum] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] InputAdapter Exchange [ctr_state] #9 WholeStageCodegen (18) - HashAggregate [ctr_state,ctr_total_return] [count,count,sum,sum] - HashAggregate [ca_state,cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,cr_returning_customer_sk] #10 + Exchange [cr_returning_customer_sk,ca_state] #10 WholeStageCodegen (17) - HashAggregate [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] [sum,sum] - Project [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] - SortMergeJoin [ca_address_sk,cr_returning_addr_sk] + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + SortMergeJoin [cr_returning_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (14) Sort [cr_returning_addr_sk] InputAdapter Exchange [cr_returning_addr_sk] #11 WholeStageCodegen (13) - Project [cr_return_amt_inc_tax,cr_returning_addr_sk,cr_returning_customer_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_return_amt_inc_tax,cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index c0587b06a0da3..2b7c6041bfb19 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip,ctr_total_return] +TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] WholeStageCodegen (11) - Project [c_customer_id,c_first_name,c_last_name,c_salutation,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip,ctr_total_return] + Project [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_customer_id,c_first_name,c_last_name,c_salutation,ctr_total_return] - BroadcastHashJoin [c_customer_sk,ctr_customer_sk] + Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,ctr_state,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] - HashAggregate [ca_state,cr_returning_customer_sk,sum] [ctr_customer_sk,ctr_state,ctr_total_return,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,cr_returning_customer_sk] #1 + Exchange [cr_returning_customer_sk,ca_state] #1 WholeStageCodegen (3) - HashAggregate [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] [sum,sum] - Project [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] - BroadcastHashJoin [ca_address_sk,cr_returning_addr_sk] - Project [cr_return_amt_inc_tax,cr_returning_addr_sk,cr_returning_customer_sk] + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_return_amt_inc_tax,cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -39,24 +39,24 @@ TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_ci BroadcastExchange #4 WholeStageCodegen (8) Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [count,ctr_state,sum] [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),avg(ctr_total_return),count,ctr_state,sum] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] InputAdapter Exchange [ctr_state] #5 WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [count,count,sum,sum] - HashAggregate [ca_state,cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] InputAdapter - Exchange [ca_state,cr_returning_customer_sk] #6 + Exchange [cr_returning_customer_sk,ca_state] #6 WholeStageCodegen (6) - HashAggregate [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] [sum,sum] - Project [ca_state,cr_return_amt_inc_tax,cr_returning_customer_sk] - BroadcastHashJoin [ca_address_sk,cr_returning_addr_sk] - Project [cr_return_amt_inc_tax,cr_returning_addr_sk,cr_returning_customer_sk] + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_return_amt_inc_tax,cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -64,14 +64,14 @@ TakeOrderedAndProject [c_customer_id,c_first_name,c_last_name,c_salutation,ca_ci InputAdapter BroadcastExchange #7 WholeStageCodegen (9) - Filter [c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_id,c_customer_sk,c_first_name,c_last_name,c_salutation] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 WholeStageCodegen (10) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_country,ca_county,ca_gmt_offset,ca_location_type,ca_state,ca_street_name,ca_street_number,ca_street_type,ca_suite_number,ca_zip] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/simplified.txt index d9f1a7056caaa..520fdde428200 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/simplified.txt @@ -1,11 +1,11 @@ -TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] WholeStageCodegen (8) - HashAggregate [i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter - Exchange [i_current_price,i_item_desc,i_item_id] #1 + Exchange [i_item_id,i_item_desc,i_current_price] #1 WholeStageCodegen (7) - HashAggregate [i_current_price,i_item_desc,i_item_id] - Project [i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter WholeStageCodegen (4) @@ -13,20 +13,20 @@ TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] InputAdapter Exchange [i_item_sk] #2 WholeStageCodegen (3) - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk,inv_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - Filter [i_current_price,i_item_sk,i_manufact_id] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + Filter [i_current_price,i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_desc,i_item_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] Project [inv_date_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + Filter [inv_quantity_on_hand,inv_item_sk,inv_date_sk] ColumnarToRow InputAdapter Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] @@ -37,7 +37,7 @@ TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (6) Sort [ss_item_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 8006c538a2a9b..92f8729caa5c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -1,26 +1,26 @@ -TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] WholeStageCodegen (5) - HashAggregate [i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter - Exchange [i_current_price,i_item_desc,i_item_id] #1 + Exchange [i_item_id,i_item_desc,i_current_price] #1 WholeStageCodegen (4) - HashAggregate [i_current_price,i_item_desc,i_item_id] - Project [i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk,inv_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [i_current_price,i_item_desc,i_item_id,i_item_sk] - Filter [i_current_price,i_item_sk,i_manufact_id] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + Filter [i_current_price,i_manufact_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_desc,i_item_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [inv_date_sk,inv_item_sk] - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand] + Filter [inv_quantity_on_hand,inv_item_sk,inv_date_sk] ColumnarToRow InputAdapter Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand] @@ -31,7 +31,7 @@ TakeOrderedAndProject [i_current_price,i_item_desc,i_item_id] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 294bc6bea32d6..7d93f4458f95e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -1,22 +1,22 @@ -TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_dev,wr_item_qty] +TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] WholeStageCodegen (18) - Project [cr_item_qty,item_id,sr_item_qty,wr_item_qty] + Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] - Project [cr_item_qty,item_id,sr_item_qty] + Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [item_id,sr_item_qty,sum,sum(cast(sr_return_quantity as bigint))] + HashAggregate [i_item_id,sum] [sum(cast(sr_return_quantity as bigint)),item_id,sr_item_qty,sum] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [i_item_id,sr_return_quantity] - BroadcastHashJoin [i_item_sk,sr_item_sk] + Project [sr_return_quantity,i_item_id] + BroadcastHashJoin [sr_item_sk,i_item_sk] Project [sr_item_sk,sr_return_quantity] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Filter [sr_item_sk,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_return_quantity] InputAdapter BroadcastExchange #2 WholeStageCodegen (3) @@ -25,7 +25,7 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -45,18 +45,18 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [cr_item_qty,item_id,sum,sum(cast(cr_return_quantity as bigint))] + HashAggregate [i_item_id,sum] [sum(cast(cr_return_quantity as bigint)),item_id,cr_item_qty,sum] InputAdapter Exchange [i_item_id] #7 WholeStageCodegen (10) - HashAggregate [cr_return_quantity,i_item_id] [sum,sum] + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] Project [cr_return_quantity,i_item_id] BroadcastHashJoin [cr_item_sk,i_item_sk] Project [cr_item_sk,cr_return_quantity] @@ -64,28 +64,28 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ Filter [cr_item_sk,cr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_item_sk,cr_return_quantity] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter BroadcastExchange #8 WholeStageCodegen (17) - HashAggregate [i_item_id,sum] [item_id,sum,sum(cast(wr_return_quantity as bigint)),wr_item_qty] + HashAggregate [i_item_id,sum] [sum(cast(wr_return_quantity as bigint)),item_id,wr_item_qty,sum] InputAdapter Exchange [i_item_id] #9 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [i_item_id,wr_return_quantity] - BroadcastHashJoin [i_item_sk,wr_item_sk] + Project [wr_return_quantity,i_item_id] + BroadcastHashJoin [wr_item_sk,i_item_sk] Project [wr_item_sk,wr_return_quantity] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_item_sk,wr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_return_quantity] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index d0639eada79ae..c12d5ffaaf914 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -1,29 +1,29 @@ -TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_dev,wr_item_qty] +TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] WholeStageCodegen (18) - Project [cr_item_qty,item_id,sr_item_qty,wr_item_qty] + Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] - Project [cr_item_qty,item_id,sr_item_qty] + Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [item_id,sr_item_qty,sum,sum(cast(sr_return_quantity as bigint))] + HashAggregate [i_item_id,sum] [sum(cast(sr_return_quantity as bigint)),item_id,sr_item_qty,sum] InputAdapter Exchange [i_item_id] #1 WholeStageCodegen (5) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [i_item_id,sr_return_quantity] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [i_item_id,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [i_item_sk,sr_item_sk] + Project [sr_return_quantity,i_item_id] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [sr_returned_date_sk,sr_return_quantity,i_item_id] + BroadcastHashJoin [sr_item_sk,i_item_sk] Filter [sr_item_sk,sr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_item_sk,sr_return_quantity] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [i_item_id,i_item_sk] + Filter [i_item_sk,i_item_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (4) @@ -32,7 +32,7 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ Filter [d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -52,40 +52,40 @@ TakeOrderedAndProject [average,cr_dev,cr_item_qty,item_id,sr_dev,sr_item_qty,wr_ InputAdapter BroadcastExchange #6 WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [cr_item_qty,item_id,sum,sum(cast(cr_return_quantity as bigint))] + HashAggregate [i_item_id,sum] [sum(cast(cr_return_quantity as bigint)),item_id,cr_item_qty,sum] InputAdapter Exchange [i_item_id] #7 WholeStageCodegen (10) - HashAggregate [cr_return_quantity,i_item_id] [sum,sum] + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] Project [cr_return_quantity,i_item_id] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] + Project [cr_returned_date_sk,cr_return_quantity,i_item_id] BroadcastHashJoin [cr_item_sk,i_item_sk] Filter [cr_item_sk,cr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_item_sk,cr_return_quantity] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 WholeStageCodegen (17) - HashAggregate [i_item_id,sum] [item_id,sum,sum(cast(wr_return_quantity as bigint)),wr_item_qty] + HashAggregate [i_item_id,sum] [sum(cast(wr_return_quantity as bigint)),item_id,wr_item_qty,sum] InputAdapter Exchange [i_item_id] #9 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [i_item_id,wr_return_quantity] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] - Project [i_item_id,wr_return_quantity,wr_returned_date_sk] - BroadcastHashJoin [i_item_sk,wr_item_sk] + Project [wr_return_quantity,i_item_id] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Project [wr_returned_date_sk,wr_return_quantity,i_item_id] + BroadcastHashJoin [wr_item_sk,i_item_sk] Filter [wr_item_sk,wr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_return_quantity] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt index 7c2afa21084ea..1fbc57ee7e47a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/simplified.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (6) - Project [c_customer_id,c_first_name,c_last_name] + Project [c_customer_id,c_last_name,c_first_name] BroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] InputAdapter BroadcastExchange #1 @@ -10,19 +10,19 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) - Project [c_current_cdemo_sk,c_customer_id,c_first_name,c_last_name] + Project [c_customer_id,c_current_cdemo_sk,c_first_name,c_last_name] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_cdemo_sk,c_current_hdemo_sk,c_customer_id,c_first_name,c_last_name] + Project [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_id,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [ca_address_sk] - Filter [ca_address_sk,ca_city] + Filter [ca_city,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_city] @@ -30,12 +30,12 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + BroadcastHashJoin [ib_income_band_sk,hd_income_band_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Project [ib_income_band_sk] - Filter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + Filter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] ColumnarToRow InputAdapter Scan parquet default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index c771df2baf5fe..014d808cf85d1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (6) - Project [c_customer_id,c_first_name,c_last_name] + Project [c_customer_id,c_last_name,c_first_name] BroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_hdemo_sk,c_customer_id,c_first_name,c_last_name,cd_demo_sk] + Project [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,c_current_hdemo_sk,c_customer_id,c_first_name,c_last_name] + Project [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_id,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #1 WholeStageCodegen (1) Project [ca_address_sk] - Filter [ca_address_sk,ca_city] + Filter [ca_city,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_city] @@ -40,7 +40,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] BroadcastExchange #4 WholeStageCodegen (4) Project [ib_income_band_sk] - Filter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + Filter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] ColumnarToRow InputAdapter Scan parquet default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 064acdf824282..ee550f1af4947 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -239,7 +239,7 @@ Arguments: [wr_refunded_cdemo_sk#15 ASC NULLS FIRST, wr_returning_cdemo_sk#17 AS Output [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_education_status), IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct (42) ColumnarToRow [codegen id : 12] @@ -247,13 +247,13 @@ Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] (43) Filter [codegen id : 12] Input [3]: [cd_demo_sk#31, cd_marital_status#32, cd_education_status#33] -Condition : ((isnotnull(cd_education_status#33) AND isnotnull(cd_marital_status#32)) AND isnotnull(cd_demo_sk#31)) +Condition : ((isnotnull(cd_demo_sk#31) AND isnotnull(cd_marital_status#32)) AND isnotnull(cd_education_status#33)) (44) Scan parquet default.customer_demographics Output [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_education_status), IsNotNull(cd_marital_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (45) ColumnarToRow [codegen id : 11] @@ -261,15 +261,15 @@ Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] (46) Filter [codegen id : 11] Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Condition : (((isnotnull(cd_demo_sk#34) AND isnotnull(cd_education_status#36)) AND isnotnull(cd_marital_status#35)) AND ((((cd_marital_status#35 = M) AND (cd_education_status#36 = Advanced Degree)) OR ((cd_marital_status#35 = S) AND (cd_education_status#36 = College))) OR ((cd_marital_status#35 = W) AND (cd_education_status#36 = 2 yr Degree)))) +Condition : (((isnotnull(cd_demo_sk#34) AND isnotnull(cd_marital_status#35)) AND isnotnull(cd_education_status#36)) AND ((((cd_marital_status#35 = M) AND (cd_education_status#36 = Advanced Degree)) OR ((cd_marital_status#35 = S) AND (cd_education_status#36 = College))) OR ((cd_marital_status#35 = W) AND (cd_education_status#36 = 2 yr Degree)))) (47) BroadcastExchange Input [3]: [cd_demo_sk#34, cd_marital_status#35, cd_education_status#36] -Arguments: HashedRelationBroadcastMode(List(input[2, string, false], input[1, string, false]),false), [id=#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, false], input[2, string, false]),false), [id=#37] (48) BroadcastHashJoin [codegen id : 12] -Left keys [2]: [cd_education_status#33, cd_marital_status#32] -Right keys [2]: [cd_education_status#36, cd_marital_status#35] +Left keys [2]: [cd_marital_status#32, cd_education_status#33] +Right keys [2]: [cd_marital_status#35, cd_education_status#36] Join condition: None (49) Project [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt index 19941ad51fc18..e7aee17172e60 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/simplified.txt @@ -1,46 +1,46 @@ -TakeOrderedAndProject [aggOrder,avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantity),substr(r_reason_desc, 1, 20)] +TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cash),avg(wr_fee),avg(ws_quantity)] WholeStageCodegen (15) - HashAggregate [count,count,count,r_reason_desc,sum,sum,sum] [aggOrder,avg(UnscaledValue(wr_fee)),avg(UnscaledValue(wr_refunded_cash)),avg(cast(ws_quantity as bigint)),avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantity),count,count,count,substr(r_reason_desc, 1, 20),sum,sum,sum] + HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(cast(ws_quantity as bigint)),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),aggOrder,sum,count,sum,count,sum,count] InputAdapter Exchange [r_reason_desc] #1 WholeStageCodegen (14) - HashAggregate [r_reason_desc,wr_fee,wr_refunded_cash,ws_quantity] [count,count,count,count,count,count,sum,sum,sum,sum,sum,sum] - Project [r_reason_desc,wr_fee,wr_refunded_cash,ws_quantity] - SortMergeJoin [cd_demo_sk,cd_demo_sk,cd_education_status,cd_marital_status,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_sales_price] + HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + SortMergeJoin [wr_refunded_cdemo_sk,wr_returning_cdemo_sk,cd_demo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] InputAdapter WholeStageCodegen (10) Sort [wr_refunded_cdemo_sk,wr_returning_cdemo_sk] InputAdapter Exchange [wr_refunded_cdemo_sk,wr_returning_cdemo_sk] #2 WholeStageCodegen (9) - Project [r_reason_desc,wr_fee,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_quantity,ws_sales_price] - BroadcastHashJoin [ca_address_sk,ca_state,wr_refunded_addr_sk,ws_net_profit] - Project [r_reason_desc,wr_fee,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_net_profit,ws_quantity,ws_sales_price] - BroadcastHashJoin [r_reason_sk,wr_reason_sk] - Project [wr_fee,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_net_profit,ws_quantity,ws_sales_price] + Project [ws_quantity,ws_sales_price,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_fee,wr_refunded_cash,r_reason_desc] + BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + Project [ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_fee,wr_refunded_cash,r_reason_desc] + BroadcastHashJoin [wr_reason_sk,r_reason_sk] + Project [ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] BroadcastHashJoin [d_date_sk,ws_sold_date_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - Project [wr_fee,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_net_profit,ws_quantity,ws_sales_price,ws_sold_date_sk] - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + Project [ws_sold_date_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] InputAdapter WholeStageCodegen (4) Sort [ws_item_sk,ws_order_number] InputAdapter Exchange [ws_item_sk,ws_order_number] #4 WholeStageCodegen (3) - Project [ws_item_sk,ws_net_profit,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] - Filter [ws_item_sk,ws_net_profit,ws_order_number,ws_sales_price,ws_sold_date_sk,ws_web_page_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sold_date_sk,ws_sales_price,ws_net_profit] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_profit,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -54,41 +54,41 @@ TakeOrderedAndProject [aggOrder,avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantit InputAdapter Exchange [wr_item_sk,wr_order_number] #6 WholeStageCodegen (5) - Filter [wr_item_sk,wr_order_number,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cdemo_sk,wr_returning_cdemo_sk] + Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_fee,wr_item_sk,wr_order_number,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk] + Scan parquet 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] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) Filter [r_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.reason [r_reason_desc,r_reason_sk] + Scan parquet default.reason [r_reason_sk,r_reason_desc] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) Project [ca_address_sk,ca_state] - Filter [ca_address_sk,ca_country,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter WholeStageCodegen (13) Sort [cd_demo_sk,cd_demo_sk] InputAdapter Exchange [cd_demo_sk,cd_demo_sk] #9 WholeStageCodegen (12) - Project [cd_demo_sk,cd_demo_sk,cd_education_status,cd_marital_status] - BroadcastHashJoin [cd_education_status,cd_education_status,cd_marital_status,cd_marital_status] - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Project [cd_demo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [cd_marital_status,cd_education_status,cd_marital_status,cd_education_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #10 WholeStageCodegen (11) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index 48a810d472642..626d1a71e579f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -151,7 +151,7 @@ Input [13]: [ws_sold_date_sk#1, ws_quantity#5, ws_sales_price#6, ws_net_profit#7 Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_education_status), IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] @@ -159,7 +159,7 @@ Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] (24) Filter [codegen id : 4] Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : ((isnotnull(cd_education_status#25) AND isnotnull(cd_demo_sk#23)) AND isnotnull(cd_marital_status#24)) +Condition : ((isnotnull(cd_demo_sk#23) AND isnotnull(cd_marital_status#24)) AND isnotnull(cd_education_status#25)) (25) BroadcastExchange Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index db6da292ac49b..93c319a615566 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [aggOrder,avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantity),substr(r_reason_desc, 1, 20)] +TakeOrderedAndProject [substr(r_reason_desc, 1, 20),aggOrder,avg(wr_refunded_cash),avg(wr_fee),avg(ws_quantity)] WholeStageCodegen (9) - HashAggregate [count,count,count,r_reason_desc,sum,sum,sum] [aggOrder,avg(UnscaledValue(wr_fee)),avg(UnscaledValue(wr_refunded_cash)),avg(cast(ws_quantity as bigint)),avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantity),count,count,count,substr(r_reason_desc, 1, 20),sum,sum,sum] + HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(cast(ws_quantity as bigint)),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),aggOrder,sum,count,sum,count,sum,count] InputAdapter Exchange [r_reason_desc] #1 WholeStageCodegen (8) - HashAggregate [r_reason_desc,wr_fee,wr_refunded_cash,ws_quantity] [count,count,count,count,count,count,sum,sum,sum,sum,sum,sum] - Project [r_reason_desc,wr_fee,wr_refunded_cash,ws_quantity] - BroadcastHashJoin [r_reason_sk,wr_reason_sk] - Project [wr_fee,wr_reason_sk,wr_refunded_cash,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_fee,wr_reason_sk,wr_refunded_cash,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [ca_address_sk,ca_state,wr_refunded_addr_sk,ws_net_profit] - Project [wr_fee,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,ws_net_profit,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cd_education_status,cd_education_status,cd_marital_status,cd_marital_status,wr_returning_cdemo_sk] - Project [cd_education_status,cd_marital_status,wr_fee,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_returning_cdemo_sk,ws_net_profit,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cd_education_status,cd_marital_status,wr_refunded_cdemo_sk,ws_sales_price] - Project [wr_fee,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_net_profit,ws_quantity,ws_sales_price,ws_sold_date_sk] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] - Project [wr_fee,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,ws_net_profit,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_web_page_sk] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [ws_item_sk,ws_net_profit,ws_order_number,ws_sales_price,ws_sold_date_sk,ws_web_page_sk] + HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + BroadcastHashJoin [wr_reason_sk,r_reason_sk] + Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + Project [ws_sold_date_sk,ws_quantity,ws_net_profit,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + Project [ws_sold_date_sk,ws_quantity,ws_net_profit,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + Project [ws_sold_date_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_sold_date_sk,ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sold_date_sk,ws_sales_price,ws_net_profit] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_profit,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [wr_item_sk,wr_order_number,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cdemo_sk,wr_returning_cdemo_sk] + Filter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_fee,wr_item_sk,wr_order_number,wr_reason_sk,wr_refunded_addr_sk,wr_refunded_cash,wr_refunded_cdemo_sk,wr_returning_cdemo_sk] + Scan parquet 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] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -40,30 +40,30 @@ TakeOrderedAndProject [aggOrder,avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantit InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_demo_sk,cd_marital_status,cd_education_status] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Project [ca_address_sk,ca_state] - Filter [ca_address_sk,ca_country,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -73,4 +73,4 @@ TakeOrderedAndProject [aggOrder,avg(wr_fee),avg(wr_refunded_cash),avg(ws_quantit Filter [r_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.reason [r_reason_desc,r_reason_sk] + Scan parquet default.reason [r_reason_sk,r_reason_desc] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/simplified.txt index 76b89dcf2f6c7..cac22cb0f4da8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_sum] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] WholeStageCodegen (6) - Project [i_category,i_class,lochierarchy,rank_within_parent,total_sum] + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w1,_w2,_w3] + Window [_w3,_w1,_w2] WholeStageCodegen (5) Sort [_w1,_w2,_w3] InputAdapter Exchange [_w1,_w2] #1 WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [_w1,_w2,_w3,lochierarchy,sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w1,_w2,_w3,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 WholeStageCodegen (3) HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [i_category,i_class,ws_net_paid] - Project [i_category,i_class,ws_net_paid] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -36,4 +36,4 @@ TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_ Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 76b89dcf2f6c7..cac22cb0f4da8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_sum] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] WholeStageCodegen (6) - Project [i_category,i_class,lochierarchy,rank_within_parent,total_sum] + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w1,_w2,_w3] + Window [_w3,_w1,_w2] WholeStageCodegen (5) Sort [_w1,_w2,_w3] InputAdapter Exchange [_w1,_w2] #1 WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [_w1,_w2,_w3,lochierarchy,sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w1,_w2,_w3,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 WholeStageCodegen (3) HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [i_category,i_class,ws_net_paid] - Project [i_category,i_class,ws_net_paid] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -36,4 +36,4 @@ TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_ Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt index 0404e73452564..8dd59340cf069 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/simplified.txt @@ -1,48 +1,48 @@ WholeStageCodegen (26) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (25) HashAggregate [count,count] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #2 + Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (24) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - SortMergeJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] - SortMergeJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + SortMergeJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] WholeStageCodegen (7) - Sort [c_first_name,c_last_name,d_date] + Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #3 + Exchange [c_last_name,c_first_name,d_date] #3 WholeStageCodegen (6) - Project [c_first_name,c_last_name,d_date] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [d_date,c_first_name,c_last_name] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (2) - Project [d_date,ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (5) Sort [c_customer_sk] @@ -54,17 +54,17 @@ WholeStageCodegen (26) InputAdapter Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name] WholeStageCodegen (15) - Sort [c_first_name,c_last_name,d_date] + Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #7 + Exchange [c_last_name,c_first_name,d_date] #7 WholeStageCodegen (14) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #8 + Exchange [c_last_name,c_first_name,d_date] #8 WholeStageCodegen (13) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (10) Sort [cs_bill_customer_sk] @@ -73,43 +73,43 @@ WholeStageCodegen (26) WholeStageCodegen (9) Project [cs_bill_customer_sk,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #5 + ReusedExchange [d_date_sk,d_date] #5 InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #6 WholeStageCodegen (23) - Sort [c_first_name,c_last_name,d_date] + Sort [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #10 + Exchange [c_last_name,c_first_name,d_date] #10 WholeStageCodegen (22) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #11 + Exchange [c_last_name,c_first_name,d_date] #11 WholeStageCodegen (21) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (18) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #12 WholeStageCodegen (17) - Project [d_date,ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #5 + ReusedExchange [d_date_sk,d_date] #5 InputAdapter WholeStageCodegen (20) Sort [c_customer_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 3caad01cb7c4e..a5b57a4ac9450 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -1,36 +1,36 @@ WholeStageCodegen (13) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (12) HashAggregate [count,count] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #2 + Exchange [c_last_name,c_first_name,d_date] #2 WholeStageCodegen (11) - HashAggregate [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] - BroadcastHashJoin [c_first_name,c_first_name,c_last_name,c_last_name,d_date,d_date] - Project [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [d_date,ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_sold_date_sk] + HashAggregate [c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + Project [d_date,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -41,40 +41,40 @@ WholeStageCodegen (13) InputAdapter BroadcastExchange #5 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #6 + Exchange [c_last_name,c_first_name,d_date] #6 WholeStageCodegen (5) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] Project [cs_bill_customer_sk,d_date] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #3 + ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter BroadcastExchange #7 WholeStageCodegen (10) - HashAggregate [c_first_name,c_last_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_first_name,c_last_name,d_date] #8 + Exchange [c_last_name,c_first_name,d_date] #8 WholeStageCodegen (9) - HashAggregate [c_first_name,c_last_name,d_date] - Project [c_first_name,c_last_name,d_date] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Project [d_date,ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_bill_customer_sk,ws_sold_date_sk] + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter - ReusedExchange [d_date,d_date_sk] #3 + ReusedExchange [d_date_sk,d_date] #3 InputAdapter ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/simplified.txt index 1e591a4bc0894..99ca86aadc661 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/simplified.txt @@ -6,21 +6,21 @@ BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin WholeStageCodegen (5) - HashAggregate [count] [count,count(1),h8_30_to_9] + HashAggregate [count] [count(1),h8_30_to_9,count] InputAdapter Exchange #1 WholeStageCodegen (4) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -28,7 +28,7 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -36,32 +36,32 @@ BroadcastNestedLoopJoin Filter [s_store_name,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] BroadcastExchange #5 WholeStageCodegen (10) - HashAggregate [count] [count,count(1),h9_to_9_30] + HashAggregate [count] [count(1),h9_to_9_30,count] InputAdapter Exchange #6 WholeStageCodegen (9) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) @@ -69,28 +69,28 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter ReusedExchange [hd_demo_sk] #4 BroadcastExchange #8 WholeStageCodegen (15) - HashAggregate [count] [count,count(1),h9_30_to_10] + HashAggregate [count] [count(1),h9_30_to_10,count] InputAdapter Exchange #9 WholeStageCodegen (14) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #10 WholeStageCodegen (11) @@ -98,28 +98,28 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter ReusedExchange [hd_demo_sk] #4 BroadcastExchange #11 WholeStageCodegen (20) - HashAggregate [count] [count,count(1),h10_to_10_30] + HashAggregate [count] [count(1),h10_to_10_30,count] InputAdapter Exchange #12 WholeStageCodegen (19) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #13 WholeStageCodegen (16) @@ -127,28 +127,28 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter ReusedExchange [hd_demo_sk] #4 BroadcastExchange #14 WholeStageCodegen (25) - HashAggregate [count] [count,count(1),h10_30_to_11] + HashAggregate [count] [count(1),h10_30_to_11,count] InputAdapter Exchange #15 WholeStageCodegen (24) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #16 WholeStageCodegen (21) @@ -156,28 +156,28 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter ReusedExchange [hd_demo_sk] #4 BroadcastExchange #17 WholeStageCodegen (30) - HashAggregate [count] [count,count(1),h11_to_11_30] + HashAggregate [count] [count(1),h11_to_11_30,count] InputAdapter Exchange #18 WholeStageCodegen (29) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #19 WholeStageCodegen (26) @@ -185,28 +185,28 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter ReusedExchange [hd_demo_sk] #4 BroadcastExchange #20 WholeStageCodegen (35) - HashAggregate [count] [count,count(1),h11_30_to_12] + HashAggregate [count] [count(1),h11_30_to_12,count] InputAdapter Exchange #21 WholeStageCodegen (34) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #22 WholeStageCodegen (31) @@ -214,28 +214,28 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter ReusedExchange [hd_demo_sk] #4 BroadcastExchange #23 WholeStageCodegen (40) - HashAggregate [count] [count,count(1),h12_to_12_30] + HashAggregate [count] [count(1),h12_to_12_30,count] InputAdapter Exchange #24 WholeStageCodegen (39) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #25 WholeStageCodegen (36) @@ -243,7 +243,7 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #3 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index 261822d0f8ce1..8e72594b02c05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -6,26 +6,26 @@ BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin WholeStageCodegen (5) - HashAggregate [count] [count,count(1),h8_30_to_9] + HashAggregate [count] [count(1),h8_30_to_9,count] InputAdapter Exchange #1 WholeStageCodegen (4) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_dep_count,hd_vehicle_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] @@ -36,7 +36,7 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -44,24 +44,24 @@ BroadcastNestedLoopJoin Filter [s_store_name,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] BroadcastExchange #5 WholeStageCodegen (10) - HashAggregate [count] [count,count(1),h9_to_9_30] + HashAggregate [count] [count(1),h9_to_9_30,count] InputAdapter Exchange #6 WholeStageCodegen (9) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -71,26 +71,26 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 BroadcastExchange #8 WholeStageCodegen (15) - HashAggregate [count] [count,count(1),h9_30_to_10] + HashAggregate [count] [count(1),h9_30_to_10,count] InputAdapter Exchange #9 WholeStageCodegen (14) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -100,26 +100,26 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 BroadcastExchange #11 WholeStageCodegen (20) - HashAggregate [count] [count,count(1),h10_to_10_30] + HashAggregate [count] [count(1),h10_to_10_30,count] InputAdapter Exchange #12 WholeStageCodegen (19) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -129,26 +129,26 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 BroadcastExchange #14 WholeStageCodegen (25) - HashAggregate [count] [count,count(1),h10_30_to_11] + HashAggregate [count] [count(1),h10_30_to_11,count] InputAdapter Exchange #15 WholeStageCodegen (24) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -158,26 +158,26 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 BroadcastExchange #17 WholeStageCodegen (30) - HashAggregate [count] [count,count(1),h11_to_11_30] + HashAggregate [count] [count(1),h11_to_11_30,count] InputAdapter Exchange #18 WholeStageCodegen (29) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -187,26 +187,26 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 BroadcastExchange #20 WholeStageCodegen (35) - HashAggregate [count] [count,count(1),h11_30_to_12] + HashAggregate [count] [count(1),h11_30_to_12,count] InputAdapter Exchange #21 WholeStageCodegen (34) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -216,26 +216,26 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 BroadcastExchange #23 WholeStageCodegen (40) - HashAggregate [count] [count,count(1),h12_to_12_30] + HashAggregate [count] [count(1),h12_to_12_30,count] InputAdapter Exchange #24 WholeStageCodegen (39) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -245,6 +245,6 @@ BroadcastNestedLoopJoin Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter ReusedExchange [s_store_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/simplified.txt index c2eb87a05109c..0d41c4559139a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/simplified.txt @@ -1,24 +1,24 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] WholeStageCodegen (7) - Project [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [_w0,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - Sort [i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name] #2 + Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,i_brand,i_category,i_class,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_category,i_class,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter BroadcastExchange #3 @@ -26,23 +26,23 @@ TakeOrderedAndProject [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_comp Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category] Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 3b3ad9cf61293..efeab7f69e1d2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] WholeStageCodegen (7) - Project [avg_monthly_sales,d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum_sales] + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [_w0,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (6) - Sort [i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 WholeStageCodegen (5) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name] #2 + Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 WholeStageCodegen (4) - HashAggregate [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,i_brand,i_category,i_class,s_company_name,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,i_brand,i_category,i_class,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_category,i_class,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk,d_moy] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt index 151ec8b405f2f..66502080dda30 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (1) @@ -14,31 +14,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #2 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #2 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #3 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #3 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #4 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #4 WholeStageCodegen (1) @@ -50,31 +50,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #5 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #5 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #6 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #6 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #7 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #7 WholeStageCodegen (1) @@ -86,31 +86,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #8 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #8 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #9 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #9 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #10 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #10 WholeStageCodegen (1) @@ -122,31 +122,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #11 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #11 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #12 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #12 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #13 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #13 WholeStageCodegen (1) @@ -158,28 +158,28 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #14 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #14 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #15 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #15 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Filter [r_reason_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt index 151ec8b405f2f..66502080dda30 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) Project Subquery #1 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (1) @@ -14,31 +14,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #2 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #2 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #3 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #3 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #4 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #4 WholeStageCodegen (1) @@ -50,31 +50,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #5 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #5 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #6 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #6 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #7 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #7 WholeStageCodegen (1) @@ -86,31 +86,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #8 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #8 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #9 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #9 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #10 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #10 WholeStageCodegen (1) @@ -122,31 +122,31 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #11 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #11 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #12 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #12 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Subquery #13 WholeStageCodegen (2) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #13 WholeStageCodegen (1) @@ -158,28 +158,28 @@ WholeStageCodegen (1) Scan parquet default.store_sales [ss_quantity] Subquery #14 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_ext_discount_amt)),avg(ss_ext_discount_amt),sum,count] InputAdapter Exchange #14 WholeStageCodegen (1) - HashAggregate [ss_ext_discount_amt] [count,count,sum,sum] + HashAggregate [ss_ext_discount_amt] [sum,count,sum,count] Project [ss_ext_discount_amt] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_discount_amt,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_ext_discount_amt] Subquery #15 WholeStageCodegen (2) - HashAggregate [count,sum] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),count,sum] + HashAggregate [sum,count] [avg(UnscaledValue(ss_net_paid)),avg(ss_net_paid),sum,count] InputAdapter Exchange #15 WholeStageCodegen (1) - HashAggregate [ss_net_paid] [count,count,sum,sum] + HashAggregate [ss_net_paid] [sum,count,sum,count] Project [ss_net_paid] Filter [ss_quantity] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_paid,ss_quantity] + Scan parquet default.store_sales [ss_quantity,ss_net_paid] Filter [r_reason_sk] ColumnarToRow InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/simplified.txt index 564b98cb6e6b1..1fe0442eab13f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/simplified.txt @@ -4,21 +4,21 @@ TakeOrderedAndProject [am_pm_ratio] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (5) - HashAggregate [count] [amc,count,count(1)] + HashAggregate [count] [count(1),amc,count] InputAdapter Exchange #1 WholeStageCodegen (4) HashAggregate [count,count] Project - BroadcastHashJoin [t_time_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] Project [ws_sold_time_sk] - BroadcastHashJoin [hd_demo_sk,ws_ship_hdemo_sk] - Project [ws_ship_hdemo_sk,ws_sold_time_sk] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + Project [ws_sold_time_sk,ws_ship_hdemo_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -26,12 +26,12 @@ TakeOrderedAndProject [am_pm_ratio] Filter [wp_char_count,wp_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_page [wp_char_count,wp_web_page_sk] + Scan parquet default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count] + Filter [hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] @@ -42,24 +42,24 @@ TakeOrderedAndProject [am_pm_ratio] Filter [t_hour,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour] BroadcastExchange #5 WholeStageCodegen (10) - HashAggregate [count] [count,count(1),pmc] + HashAggregate [count] [count(1),pmc,count] InputAdapter Exchange #6 WholeStageCodegen (9) HashAggregate [count,count] Project - BroadcastHashJoin [t_time_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] Project [ws_sold_time_sk] - BroadcastHashJoin [hd_demo_sk,ws_ship_hdemo_sk] - Project [ws_ship_hdemo_sk,ws_sold_time_sk] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + Project [ws_sold_time_sk,ws_ship_hdemo_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] InputAdapter ReusedExchange [wp_web_page_sk] #2 InputAdapter @@ -71,4 +71,4 @@ TakeOrderedAndProject [am_pm_ratio] Filter [t_hour,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index 5c0047a2f2ae0..121d84d9dde2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -4,26 +4,26 @@ TakeOrderedAndProject [am_pm_ratio] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (5) - HashAggregate [count] [amc,count,count(1)] + HashAggregate [count] [count(1),amc,count] InputAdapter Exchange #1 WholeStageCodegen (4) HashAggregate [count,count] Project - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] Project [ws_web_page_sk] - BroadcastHashJoin [t_time_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [hd_demo_sk,ws_ship_hdemo_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count] + Filter [hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] @@ -34,7 +34,7 @@ TakeOrderedAndProject [am_pm_ratio] Filter [t_hour,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -42,24 +42,24 @@ TakeOrderedAndProject [am_pm_ratio] Filter [wp_char_count,wp_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_page [wp_char_count,wp_web_page_sk] + Scan parquet default.web_page [wp_web_page_sk,wp_char_count] BroadcastExchange #5 WholeStageCodegen (10) - HashAggregate [count] [count,count(1),pmc] + HashAggregate [count] [count(1),pmc,count] InputAdapter Exchange #6 WholeStageCodegen (9) HashAggregate [count,count] Project - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] Project [ws_web_page_sk] - BroadcastHashJoin [t_time_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [hd_demo_sk,ws_ship_hdemo_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] Filter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] InputAdapter ReusedExchange [hd_demo_sk] #2 InputAdapter @@ -69,6 +69,6 @@ TakeOrderedAndProject [am_pm_ratio] Filter [t_hour,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour] InputAdapter ReusedExchange [wp_web_page_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index 7ab928267ae7b..f64791821893d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -3,32 +3,32 @@ WholeStageCodegen (9) InputAdapter Exchange [Returns_Loss] #1 WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,sum,sum(UnscaledValue(cr_net_loss))] + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] InputAdapter - Exchange [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status] #2 + Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,cr_net_loss] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - Project [cd_education_status,cd_marital_status,cr_call_center_sk,cr_net_loss] + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] + Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [c_customer_sk,cd_education_status,cd_marital_status] + Project [cd_marital_status,cd_education_status,c_customer_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_customer_sk,cd_education_status,cd_marital_status] + Project [cd_marital_status,cd_education_status,c_customer_sk,c_current_addr_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,c_customer_sk,cd_education_status,cd_marital_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cd_marital_status,cd_education_status,c_customer_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cd_demo_sk,c_current_cdemo_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,36 +36,36 @@ WholeStageCodegen (9) Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Project [cr_call_center_sk,cr_net_loss,cr_returning_customer_sk] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + BroadcastHashJoin [d_date_sk,cr_returned_date_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_call_center_sk,cr_net_loss,cr_returned_date_sk,cr_returning_customer_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) Filter [cc_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.call_center [cc_call_center_id,cc_call_center_sk,cc_manager,cc_name] + Scan parquet default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index 2d55e53e02ccc..58ebe15d1750f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -3,64 +3,64 @@ WholeStageCodegen (9) InputAdapter Exchange [Returns_Loss] #1 WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,sum,sum(UnscaledValue(cr_net_loss))] + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] InputAdapter - Exchange [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status] #2 + Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,cr_net_loss] + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_hdemo_sk,cc_call_center_id,cc_manager,cc_name,cd_education_status,cd_marital_status,cr_net_loss] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,c_current_hdemo_sk,cc_call_center_id,cc_manager,cc_name,cr_net_loss] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cc_call_center_id,cc_manager,cc_name,cr_net_loss] - BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [cc_call_center_id,cc_manager,cc_name,cr_net_loss,cr_returning_customer_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_manager,cc_name,cr_net_loss,cr_returned_date_sk,cr_returning_customer_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returned_date_sk,cr_returning_customer_sk,cr_net_loss] BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] Filter [cc_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.call_center [cc_call_center_id,cc_call_center_sk,cc_manager,cc_name] + Scan parquet default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_call_center_sk,cr_net_loss,cr_returned_date_sk,cr_returning_customer_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Project [ca_address_sk] - Filter [ca_address_sk,ca_gmt_offset] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - Filter [cd_demo_sk,cd_education_status,cd_marital_status] + Filter [cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) @@ -68,4 +68,4 @@ WholeStageCodegen (9) Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index d1713689175aa..7fd1cd3637a09 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -1,39 +1,39 @@ TakeOrderedAndProject [Excess Discount Amount ] WholeStageCodegen (7) - HashAggregate [sum] [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] + HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] InputAdapter Exchange #1 WholeStageCodegen (6) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),i_item_sk,ws_ext_discount_amt,ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_discount_amt] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] InputAdapter BroadcastExchange #2 WholeStageCodegen (4) - Project [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),i_item_sk] + Project [i_item_sk,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] BroadcastHashJoin [i_item_sk,ws_item_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [i_item_sk] - Filter [i_item_sk,i_manufact_id] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] - HashAggregate [count,sum,ws_item_sk] [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),avg(UnscaledValue(ws_ext_discount_amt)),count,sum,ws_item_sk] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),ws_item_sk,sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) - HashAggregate [ws_ext_discount_amt,ws_item_sk] [count,count,sum,sum] - Project [ws_ext_discount_amt,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + Project [ws_item_sk,ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_discount_amt,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_discount_amt] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -41,10 +41,10 @@ TakeOrderedAndProject [Excess Discount Amount ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] - Filter [ws_ext_discount_amt,ws_item_sk,ws_sold_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] + Filter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_discount_amt,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_discount_amt] InputAdapter ReusedExchange [d_date_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 98daa0669d83a..652b2e36cf781 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [Excess Discount Amount ] WholeStageCodegen (7) - HashAggregate [sum] [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] + HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] InputAdapter Exchange #1 WholeStageCodegen (6) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),i_item_sk,ws_ext_discount_amt,ws_item_sk] - Project [i_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Filter [ws_ext_discount_amt,ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_discount_amt] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] + Project [ws_sold_date_sk,ws_ext_discount_amt,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_discount_amt,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_discount_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [i_item_sk] - Filter [i_item_sk,i_manufact_id] + Filter [i_manufact_id,i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] @@ -27,17 +27,17 @@ TakeOrderedAndProject [Excess Discount Amount ] BroadcastExchange #3 WholeStageCodegen (4) Filter [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6)))] - HashAggregate [count,sum,ws_item_sk] [(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),avg(UnscaledValue(ws_ext_discount_amt)),count,sum,ws_item_sk] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(CAST(1.3 AS DECIMAL(11,6)) * CAST(avg(ws_ext_discount_amt) AS DECIMAL(11,6))),ws_item_sk,sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) - HashAggregate [ws_ext_discount_amt,ws_item_sk] [count,count,sum,sum] - Project [ws_ext_discount_amt,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + Project [ws_item_sk,ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_discount_amt,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_discount_amt] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -45,6 +45,6 @@ TakeOrderedAndProject [Excess Discount Amount ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt index e1144076af8c6..bb3eb6cb9f2de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/simplified.txt @@ -1,11 +1,11 @@ -TakeOrderedAndProject [ss_customer_sk,sumsales] +TakeOrderedAndProject [sumsales,ss_customer_sk] WholeStageCodegen (7) - HashAggregate [isEmpty,ss_customer_sk,sum] [isEmpty,sum,sum(act_sales),sumsales] + HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] InputAdapter Exchange [ss_customer_sk] #1 WholeStageCodegen (6) - HashAggregate [act_sales,ss_customer_sk] [isEmpty,isEmpty,sum,sum] - Project [sr_return_quantity,ss_customer_sk,ss_quantity,ss_sales_price] + HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] + Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] InputAdapter WholeStageCodegen (3) @@ -13,12 +13,12 @@ TakeOrderedAndProject [ss_customer_sk,sumsales] InputAdapter Exchange [sr_item_sk,sr_ticket_number] #2 WholeStageCodegen (2) - Project [sr_item_sk,sr_return_quantity,sr_ticket_number] - BroadcastHashJoin [r_reason_sk,sr_reason_sk] - Filter [sr_item_sk,sr_reason_sk,sr_ticket_number] + Project [sr_item_sk,sr_ticket_number,sr_return_quantity] + BroadcastHashJoin [sr_reason_sk,r_reason_sk] + Filter [sr_item_sk,sr_ticket_number,sr_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_reason_sk,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject [ss_customer_sk,sumsales] Filter [r_reason_desc,r_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.reason [r_reason_desc,r_reason_sk] + Scan parquet default.reason [r_reason_sk,r_reason_desc] InputAdapter WholeStageCodegen (5) Sort [ss_item_sk,ss_ticket_number] @@ -35,4 +35,4 @@ TakeOrderedAndProject [ss_customer_sk,sumsales] WholeStageCodegen (4) ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt index 05e17cc4b3bd8..81de31a443785 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt @@ -1,24 +1,24 @@ -TakeOrderedAndProject [ss_customer_sk,sumsales] +TakeOrderedAndProject [sumsales,ss_customer_sk] WholeStageCodegen (4) - HashAggregate [isEmpty,ss_customer_sk,sum] [isEmpty,sum,sum(act_sales),sumsales] + HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] InputAdapter Exchange [ss_customer_sk] #1 WholeStageCodegen (3) - HashAggregate [act_sales,ss_customer_sk] [isEmpty,isEmpty,sum,sum] - Project [sr_return_quantity,ss_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [r_reason_sk,sr_reason_sk] - Project [sr_reason_sk,sr_return_quantity,ss_customer_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] + Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] + BroadcastHashJoin [sr_reason_sk,r_reason_sk] + Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_item_sk,sr_reason_sk,sr_ticket_number] + Filter [sr_item_sk,sr_ticket_number,sr_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_reason_sk,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -26,4 +26,4 @@ TakeOrderedAndProject [ss_customer_sk,sumsales] Filter [r_reason_desc,r_reason_sk] ColumnarToRow InputAdapter - Scan parquet default.reason [r_reason_desc,r_reason_sk] + Scan parquet default.reason [r_reason_sk,r_reason_desc] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt index 95ff0dbb4ad50..128a8179ac10b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/simplified.txt @@ -1,29 +1,29 @@ -TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] +TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] WholeStageCodegen (14) - HashAggregate [count,sum,sum] [count,count(ws_order_number),order count ,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),total net profit ,total shipping cost ] + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 WholeStageCodegen (13) - HashAggregate [ws_order_number] [count,count,count(ws_order_number),sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - HashAggregate [ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] InputAdapter Exchange [ws_order_number] #2 WholeStageCodegen (12) - HashAggregate [ws_ext_ship_cost,ws_net_profit,ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_date_sk] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_date_sk,ws_web_site_sk] - BroadcastHashJoin [ca_address_sk,ws_ship_addr_sk] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] InputAdapter - SortMergeJoin [wr_order_number,ws_order_number] + SortMergeJoin [ws_order_number,wr_order_number] WholeStageCodegen (6) Sort [ws_order_number] InputAdapter Exchange [ws_order_number] #3 WholeStageCodegen (5) - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] InputAdapter SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] WholeStageCodegen (2) @@ -31,19 +31,19 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] InputAdapter Exchange [ws_order_number] #4 WholeStageCodegen (1) - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_warehouse_sk,ws_web_site_sk] + Scan parquet 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] WholeStageCodegen (4) Sort [ws_order_number] InputAdapter Exchange [ws_order_number] #5 WholeStageCodegen (3) - Project [ws_order_number,ws_warehouse_sk] + Project [ws_warehouse_sk,ws_order_number] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] WholeStageCodegen (8) Sort [wr_order_number] InputAdapter @@ -56,7 +56,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #7 WholeStageCodegen (9) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -67,7 +67,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_company_name,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_company_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (11) @@ -75,4 +75,4 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt index fda5a9f037ad4..9d30b998fe174 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt @@ -1,35 +1,35 @@ -TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] +TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] WholeStageCodegen (8) - HashAggregate [count,sum,sum] [count,count(ws_order_number),order count ,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),total net profit ,total shipping cost ] + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 WholeStageCodegen (7) - HashAggregate [ws_order_number] [count,count,count(ws_order_number),sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - HashAggregate [ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] InputAdapter Exchange [ws_order_number] #2 WholeStageCodegen (6) - HashAggregate [ws_ext_ship_cost,ws_net_profit,ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_web_site_sk] - BroadcastHashJoin [ca_address_sk,ws_ship_addr_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_web_site_sk] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] - BroadcastHashJoin [wr_order_number,ws_order_number] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + BroadcastHashJoin [ws_order_number,wr_order_number] + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] BroadcastHashJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_warehouse_sk,ws_web_site_sk] + Scan parquet 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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [ws_order_number,ws_warehouse_sk] + Project [ws_warehouse_sk,ws_order_number] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -43,12 +43,12 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -59,4 +59,4 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_company_name,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt index 420cd6f89b02a..bdcbb87b372dc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/simplified.txt @@ -1,23 +1,23 @@ -TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] +TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] WholeStageCodegen (24) - HashAggregate [count,sum,sum] [count,count(ws_order_number),order count ,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),total net profit ,total shipping cost ] + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 WholeStageCodegen (23) - HashAggregate [ws_order_number] [count,count,count(ws_order_number),sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - HashAggregate [ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] InputAdapter Exchange [ws_order_number] #2 WholeStageCodegen (22) - HashAggregate [ws_ext_ship_cost,ws_net_profit,ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_date_sk] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_date_sk,ws_web_site_sk] - BroadcastHashJoin [ca_address_sk,ws_ship_addr_sk] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] InputAdapter - SortMergeJoin [wr_order_number,ws_order_number] + SortMergeJoin [ws_order_number,wr_order_number] WholeStageCodegen (9) Sort [ws_order_number] InputAdapter @@ -28,10 +28,10 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] InputAdapter Exchange [ws_order_number] #4 WholeStageCodegen (1) - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Scan parquet 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] WholeStageCodegen (8) Sort [ws_order_number] InputAdapter @@ -48,15 +48,15 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] InputAdapter WholeStageCodegen (6) Sort [ws_order_number] InputAdapter - ReusedExchange [ws_order_number,ws_warehouse_sk] #6 + ReusedExchange [ws_warehouse_sk,ws_order_number] #6 WholeStageCodegen (18) Project [wr_order_number] - SortMergeJoin [wr_order_number,ws_order_number] + SortMergeJoin [ws_order_number,wr_order_number] InputAdapter WholeStageCodegen (15) Sort [ws_order_number] @@ -69,12 +69,12 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] WholeStageCodegen (11) Sort [ws_order_number] InputAdapter - ReusedExchange [ws_order_number,ws_warehouse_sk] #6 + ReusedExchange [ws_warehouse_sk,ws_order_number] #6 InputAdapter WholeStageCodegen (13) Sort [ws_order_number] InputAdapter - ReusedExchange [ws_order_number,ws_warehouse_sk] #6 + ReusedExchange [ws_warehouse_sk,ws_order_number] #6 InputAdapter WholeStageCodegen (17) Sort [wr_order_number] @@ -89,7 +89,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] BroadcastExchange #9 WholeStageCodegen (19) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -100,7 +100,7 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_company_name,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_company_name] InputAdapter BroadcastExchange #11 WholeStageCodegen (21) @@ -108,4 +108,4 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt index 00f1e8cc15922..6d35311c810f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt @@ -1,27 +1,27 @@ -TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] +TakeOrderedAndProject [order count ,total shipping cost ,total net profit ] WholeStageCodegen (11) - HashAggregate [count,sum,sum] [count,count(ws_order_number),order count ,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),total net profit ,total shipping cost ] + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] InputAdapter Exchange #1 WholeStageCodegen (10) - HashAggregate [ws_order_number] [count,count,count(ws_order_number),sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - HashAggregate [ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] InputAdapter Exchange [ws_order_number] #2 WholeStageCodegen (9) - HashAggregate [ws_ext_ship_cost,ws_net_profit,ws_order_number] [sum,sum,sum,sum,sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_web_site_sk] - BroadcastHashJoin [ca_address_sk,ws_ship_addr_sk] - Project [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_web_site_sk] - BroadcastHashJoin [d_date_sk,ws_ship_date_sk] - BroadcastHashJoin [wr_order_number,ws_order_number] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + BroadcastHashJoin [ws_order_number,wr_order_number] BroadcastHashJoin [ws_order_number,ws_order_number] - Filter [ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Filter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_ship_addr_sk,ws_ship_date_sk,ws_web_site_sk] + Scan parquet 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] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -30,14 +30,14 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Filter [ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] InputAdapter BroadcastExchange #5 WholeStageCodegen (5) @@ -55,9 +55,9 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [ws_order_number,ws_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_order_number,ws_warehouse_sk] + Scan parquet default.web_sales [ws_warehouse_sk,ws_order_number] InputAdapter - ReusedExchange [ws_order_number,ws_warehouse_sk] #4 + ReusedExchange [ws_warehouse_sk,ws_order_number] #4 InputAdapter BroadcastExchange #7 WholeStageCodegen (6) @@ -65,12 +65,12 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -81,4 +81,4 @@ TakeOrderedAndProject [order count ,total net profit ,total shipping cost ] Filter [web_company_name,web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_company_name,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_company_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.sf100/simplified.txt index 68a1f4798d933..1355caffbbfe8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.sf100/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [count(1)] WholeStageCodegen (5) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (4) HashAggregate [count,count] Project - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_hdemo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_hdemo_sk,ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) @@ -22,7 +22,7 @@ TakeOrderedAndProject [count(1)] Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -30,12 +30,12 @@ TakeOrderedAndProject [count(1)] Filter [s_store_name,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count] + Filter [hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index d2b7ff2fdc2c2..b13f28bf69cfd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [count(1)] WholeStageCodegen (5) - HashAggregate [count] [count,count(1),count(1)] + HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 WholeStageCodegen (4) HashAggregate [count,count] Project - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk] BroadcastHashJoin [ss_sold_time_sk,t_time_sk] Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Filter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [hd_demo_sk] - Filter [hd_demo_sk,hd_dep_count] + Filter [hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_dep_count] @@ -30,7 +30,7 @@ TakeOrderedAndProject [count(1)] Filter [t_hour,t_minute,t_time_sk] ColumnarToRow InputAdapter - Scan parquet default.time_dim [t_hour,t_minute,t_time_sk] + Scan parquet default.time_dim [t_time_sk,t_hour,t_minute] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -38,4 +38,4 @@ TakeOrderedAndProject [count(1)] Filter [s_store_name,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/simplified.txt index 7d6b96e705c39..bae48ec244faa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/simplified.txt @@ -1,13 +1,13 @@ CollectLimit WholeStageCodegen (8) - HashAggregate [sum,sum,sum] [catalog_only,store_and_catalog,store_only,sum,sum,sum,sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint))] + HashAggregate [sum,sum,sum] [sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),store_only,catalog_only,store_and_catalog,sum,sum,sum] InputAdapter Exchange #1 WholeStageCodegen (7) HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] Project [customer_sk,customer_sk] InputAdapter - SortMergeJoin [customer_sk,customer_sk,item_sk,item_sk] + SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] WholeStageCodegen (3) Sort [customer_sk,item_sk] HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] @@ -15,17 +15,17 @@ CollectLimit Exchange [ss_customer_sk,ss_item_sk] #2 WholeStageCodegen (2) HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_customer_sk,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -41,6 +41,6 @@ CollectLimit Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index 7d6b96e705c39..bae48ec244faa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -1,13 +1,13 @@ CollectLimit WholeStageCodegen (8) - HashAggregate [sum,sum,sum] [catalog_only,store_and_catalog,store_only,sum,sum,sum,sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint))] + HashAggregate [sum,sum,sum] [sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END as bigint)),store_only,catalog_only,store_and_catalog,sum,sum,sum] InputAdapter Exchange #1 WholeStageCodegen (7) HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] Project [customer_sk,customer_sk] InputAdapter - SortMergeJoin [customer_sk,customer_sk,item_sk,item_sk] + SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] WholeStageCodegen (3) Sort [customer_sk,item_sk] HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] @@ -15,17 +15,17 @@ CollectLimit Exchange [ss_customer_sk,ss_item_sk] #2 WholeStageCodegen (2) HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_customer_sk,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -41,6 +41,6 @@ CollectLimit Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt index 8e55b010434aa..a180ed2b060b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (10) - Project [i_category,i_class,i_current_price,i_item_desc,itemrevenue,revenueratio] - Sort [i_category,i_class,i_item_desc,i_item_id,revenueratio] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - Exchange [i_category,i_class,i_item_desc,i_item_id,revenueratio] #1 + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -12,25 +12,25 @@ WholeStageCodegen (10) InputAdapter Exchange [i_class] #2 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #3 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (6) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (2) - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -38,7 +38,7 @@ WholeStageCodegen (10) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -48,4 +48,4 @@ WholeStageCodegen (10) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 6f39176d9a9e1..2af7122366ebe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (7) - Project [i_category,i_class,i_current_price,i_item_desc,itemrevenue,revenueratio] - Sort [i_category,i_class,i_item_desc,i_item_id,revenueratio] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - Exchange [i_category,i_class,i_item_desc,i_item_id,revenueratio] #1 + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -12,26 +12,26 @@ WholeStageCodegen (7) InputAdapter Exchange [i_class] #2 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #3 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (3) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -39,4 +39,4 @@ WholeStageCodegen (7) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt index 0f3000ced5421..de3b1913ae25c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/simplified.txt @@ -1,30 +1,30 @@ -TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,cc_name,sm_type,substr(w_warehouse_name, 1, 20)] +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) - HashAggregate [cc_name,sm_type,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint))] + HashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint)),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter - Exchange [cc_name,sm_type,substr(w_warehouse_name, 1, 20)] #1 + Exchange [substr(w_warehouse_name, 1, 20),sm_type,cc_name] #1 WholeStageCodegen (5) - HashAggregate [cc_name,cs_ship_date_sk,cs_sold_date_sk,sm_type,w_warehouse_name] [substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cc_name,cs_ship_date_sk,cs_sold_date_sk,sm_type,w_warehouse_name] + HashAggregate [w_warehouse_name,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] + Project [cs_sold_date_sk,cs_ship_date_sk,w_warehouse_name,sm_type,cc_name] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Project [cc_name,cs_ship_date_sk,cs_sold_date_sk,cs_warehouse_sk,sm_type] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_ship_date_sk,cs_sold_date_sk,cs_warehouse_sk,sm_type] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_warehouse_sk,sm_type,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_warehouse_sk,sm_type] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_call_center_sk,cs_ship_date_sk,cs_ship_mode_sk,cs_sold_date_sk,cs_warehouse_sk] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] + BroadcastHashJoin [d_date_sk,cs_ship_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] - Filter [cs_call_center_sk,cs_ship_date_sk,cs_ship_mode_sk,cs_warehouse_sk] + Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ship_date_sk,cs_ship_mode_sk,cs_sold_date_sk,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -45,4 +45,4 @@ TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index 1cb20efd6fc34..9ebaaac52930a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -1,29 +1,29 @@ -TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,cc_name,sm_type,substr(w_warehouse_name, 1, 20)] +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] WholeStageCodegen (6) - HashAggregate [cc_name,sm_type,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint))] + HashAggregate [substr(w_warehouse_name, 1, 20),sm_type,cc_name,sum,sum,sum,sum,sum] [sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END as bigint)),sum(cast(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END as bigint)),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter - Exchange [cc_name,sm_type,substr(w_warehouse_name, 1, 20)] #1 + Exchange [substr(w_warehouse_name, 1, 20),sm_type,cc_name] #1 WholeStageCodegen (5) - HashAggregate [cc_name,cs_ship_date_sk,cs_sold_date_sk,sm_type,w_warehouse_name] [substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cc_name,cs_ship_date_sk,cs_sold_date_sk,sm_type,w_warehouse_name] + HashAggregate [w_warehouse_name,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,substr(w_warehouse_name, 1, 20),sum,sum,sum,sum,sum] + Project [cs_sold_date_sk,cs_ship_date_sk,w_warehouse_name,sm_type,cc_name] BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cc_name,cs_ship_date_sk,cs_sold_date_sk,sm_type,w_warehouse_name] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_ship_date_sk,cs_sold_date_sk,sm_type,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,w_warehouse_name,sm_type,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,w_warehouse_name,sm_type] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_call_center_sk,cs_ship_date_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,w_warehouse_name] BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_call_center_sk,cs_ship_date_sk,cs_ship_mode_sk,cs_warehouse_sk] + Filter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ship_date_sk,cs_ship_mode_sk,cs_sold_date_sk,cs_warehouse_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) @@ -42,7 +42,7 @@ TakeOrderedAndProject [30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 BroadcastExchange #5 WholeStageCodegen (4) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index f0aa032e0f2b6..1146a80d45727 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -93,7 +93,7 @@ Condition : isnotnull(ss_sold_date_sk#5) Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 3] @@ -101,7 +101,7 @@ Input [3]: [d_date_sk#7, d_year#8, d_moy#9] (11) Filter [codegen id : 3] Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_moy#9) AND isnotnull(d_year#8)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) (12) Project [codegen id : 3] Output [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt index 9a144aee5f12d..f9276bf1058fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/simplified.txt @@ -1,18 +1,18 @@ -TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] +TakeOrderedAndProject [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,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (14) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count,count(1)] + HashAggregate [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] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter - Exchange [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [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] #1 WholeStageCodegen (13) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [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,count] + Project [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] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (12) Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] InputAdapter SortMergeJoin [c_customer_sk,customer_sk] SortMergeJoin [c_customer_sk,ss_customer_sk] @@ -24,26 +24,26 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #4 WholeStageCodegen (4) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (10) Sort [customer_sk] InputAdapter @@ -51,11 +51,11 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Union WholeStageCodegen (7) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (9) @@ -64,18 +64,18 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter BroadcastExchange #7 WholeStageCodegen (11) Project [ca_address_sk] - Filter [ca_address_sk,ca_county] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 182e040762cdb..40d3299de4ac9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -81,7 +81,7 @@ Condition : isnotnull(ss_sold_date_sk#4) Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] @@ -89,7 +89,7 @@ Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (9) Filter [codegen id : 1] Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : (((((isnotnull(d_moy#8) AND isnotnull(d_year#7)) AND (d_year#7 = 2002)) AND (d_moy#8 >= 4)) AND (d_moy#8 <= 7)) AND isnotnull(d_date_sk#6)) +Condition : (((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2002)) AND (d_moy#8 >= 4)) AND (d_moy#8 <= 7)) AND isnotnull(d_date_sk#6)) (10) Project [codegen id : 1] Output [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 90105ffd73ea8..c4567e89049ac 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] +TakeOrderedAndProject [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,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] WholeStageCodegen (10) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate,count] [cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count,count(1)] + HashAggregate [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] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter - Exchange [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] #1 + Exchange [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] #1 WholeStageCodegen (9) - HashAggregate [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] [count,count] - Project [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + HashAggregate [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,count] + Project [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] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 Union WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) @@ -51,14 +51,14 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 WholeStageCodegen (7) Project [ca_address_sk] - Filter [ca_address_sk,ca_county] + Filter [ca_county,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_county] @@ -68,4 +68,4 @@ TakeOrderedAndProject [cd_credit_rating,cd_dep_college_count,cd_dep_count,cd_dep Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_credit_rating,cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_education_status,cd_gender,cd_marital_status,cd_purchase_estimate] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt index 2481e9900c5f6..93d3d21e1e02f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt @@ -1,10 +1,10 @@ -TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,customer_last_name] +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (35) - Project [customer_email_address,customer_first_name,customer_id,customer_last_name] + Project [customer_id,customer_first_name,customer_last_name,customer_email_address] SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) @@ -16,29 +16,29 @@ TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,cu Exchange [customer_id] #1 WholeStageCodegen (7) Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #2 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #2 WholeStageCodegen (6) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #3 WholeStageCodegen (2) - Project [d_year,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -48,39 +48,39 @@ TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,cu InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (4) - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter WholeStageCodegen (16) Sort [customer_id] InputAdapter Exchange [customer_id] #6 WholeStageCodegen (15) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_email_address,customer_first_name,customer_id,customer_last_name,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #7 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #7 WholeStageCodegen (14) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (10) - Project [d_year,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -88,7 +88,7 @@ TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,cu WholeStageCodegen (13) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (25) Sort [customer_id] @@ -97,61 +97,61 @@ TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,cu WholeStageCodegen (24) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #11 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (23) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (20) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #12 WholeStageCodegen (19) - Project [d_year,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #14 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 WholeStageCodegen (32) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + HashAggregate [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] [sum,sum] + Project [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] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (29) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 WholeStageCodegen (28) - Project [d_year,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #9 InputAdapter WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 4369a86a18237..ab72d5041eeea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -1,60 +1,60 @@ -TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,customer_last_name] +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] WholeStageCodegen (16) - Project [customer_email_address,customer_first_name,customer_id,customer_last_name] + Project [customer_id,customer_first_name,customer_last_name,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_email_address,customer_first_name,customer_id,customer_id,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #1 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 WholeStageCodegen (3) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_list_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_email_address,customer_first_name,customer_id,customer_last_name,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #5 + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 WholeStageCodegen (6) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ss_ext_discount_amt,ss_ext_list_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_sold_date_sk,ss_ext_discount_amt,ss_ext_list_price] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -63,45 +63,45 @@ TakeOrderedAndProject [customer_email_address,customer_first_name,customer_id,cu WholeStageCodegen (11) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #8 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #8 WholeStageCodegen (10) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_list_price] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #10 WholeStageCodegen (15) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,sum] [customer_id,sum,sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2), true))),customer_id,year_total,sum] InputAdapter - Exchange [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year] #11 + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 WholeStageCodegen (14) - HashAggregate [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] [sum,sum] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,d_year,ws_ext_discount_amt,ws_ext_list_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_birth_country,c_customer_id,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + HashAggregate [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] [sum,sum] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_sold_date_sk,ws_ext_discount_amt,ws_ext_list_price] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_customer_id,c_customer_sk,c_email_address,c_first_name,c_last_name,c_login,c_preferred_cust_flag] + Scan parquet 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] InputAdapter - ReusedExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #9 + ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt index cf472842f1431..d8db515f84f68 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -8,25 +8,25 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ws_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (6) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] - SortMergeJoin [i_item_sk,ws_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (2) - Project [ws_ext_sales_price,ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) @@ -34,7 +34,7 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -44,4 +44,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 620baa8d07fa8..57e8b40433b84 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -8,26 +8,26 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ws_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (3) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ws_ext_sales_price,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -35,4 +35,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 583f88961a836..25a1ca79cc500 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -137,7 +137,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -145,7 +145,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -196,7 +196,7 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, d_date_sk#10] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category_id), IsNotNull(i_class_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] @@ -204,7 +204,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (21) Filter [codegen id : 4] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) AND isnotnull(i_brand_id#7)) +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (22) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] @@ -437,7 +437,7 @@ Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_quantity#3, ss_list_price#4, d_d Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category_id), IsNotNull(i_class_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (73) ColumnarToRow [codegen id : 20] @@ -445,7 +445,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (74) Filter [codegen id : 20] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) AND isnotnull(i_brand_id#7)) +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (75) Exchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] @@ -766,7 +766,7 @@ Subquery:2 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquer Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_dom), IsNotNull(d_year), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] @@ -774,7 +774,7 @@ Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] (139) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Condition : (((((isnotnull(d_moy#89) AND isnotnull(d_dom#90)) AND isnotnull(d_year#11)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1999)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) (140) Project [codegen id : 1] Output [1]: [d_week_seq#29] @@ -793,7 +793,7 @@ Subquery:4 Hosting operator id = 95 Hosting Expression = Subquery scalar-subquer Output [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_dom), IsNotNull(d_year), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (142) ColumnarToRow [codegen id : 1] @@ -801,7 +801,7 @@ Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] (143) Filter [codegen id : 1] Input [4]: [d_week_seq#29, d_year#11, d_moy#89, d_dom#90] -Condition : (((((isnotnull(d_moy#89) AND isnotnull(d_dom#90)) AND isnotnull(d_year#11)) AND (d_year#11 = 1998)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#89)) AND isnotnull(d_dom#90)) AND (d_year#11 = 1998)) AND (d_moy#89 = 12)) AND (d_dom#90 = 16)) (144) Project [codegen id : 1] Output [1]: [d_week_seq#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 5141d40e7325b..37186560cb3b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,59 +1,59 @@ -TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,number_sales,number_sales,sales,sales] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (78) - BroadcastHashJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id] - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #15 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [d_date_sk] #16 WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #16 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #1 + Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (38) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_item_sk,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] WholeStageCodegen (2) @@ -64,72 +64,72 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] WholeStageCodegen (18) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #3 WholeStageCodegen (17) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (16) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #5 + Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (15) - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (6) - Sort [brand_id,category_id,class_id] + Sort [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #6 + Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #7 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (10) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #9 + Exchange [i_brand_id,i_class_id,i_category_id] #9 WholeStageCodegen (9) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter @@ -138,36 +138,36 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (14) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #11 + Exchange [i_brand_id,i_class_id,i_category_id] #11 WholeStageCodegen (13) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #10 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #12 WholeStageCodegen (19) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #1 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] @@ -179,10 +179,10 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat InputAdapter Exchange [i_item_sk] #14 WholeStageCodegen (20) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (37) Sort [ss_item_sk] InputAdapter @@ -190,24 +190,24 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat InputAdapter BroadcastExchange #17 WholeStageCodegen (77) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #18 + Exchange [i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (76) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_item_sk,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] WholeStageCodegen (40) Sort [ss_item_sk] InputAdapter - ReusedExchange [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] #2 WholeStageCodegen (56) Sort [ss_item_sk] InputAdapter @@ -216,16 +216,16 @@ TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_cat BroadcastExchange #19 WholeStageCodegen (57) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #3 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 983e96981c031..ea0e8319f3fe0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -119,7 +119,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -127,7 +127,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) +Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -147,7 +147,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 1] @@ -155,7 +155,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (12) Filter [codegen id : 1] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_category_id#8)) AND isnotnull(i_class_id#7)) +Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (13) BroadcastExchange Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] @@ -368,7 +368,7 @@ Join condition: None Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] @@ -376,7 +376,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (60) Filter [codegen id : 23] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) +Condition : (((isnotnull(i_item_sk#5) AND isnotnull(i_brand_id#6)) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (61) ReusedExchange [Reuses operator id: 56] Output [1]: [ss_item_sk#25] @@ -719,7 +719,7 @@ Subquery:2 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] @@ -727,7 +727,7 @@ Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] (129) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Condition : (((((isnotnull(d_dom#87) AND isnotnull(d_moy#86)) AND isnotnull(d_year#11)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1999)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) (130) Project [codegen id : 1] Output [1]: [d_week_seq#28] @@ -746,7 +746,7 @@ Subquery:4 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquer Output [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (132) ColumnarToRow [codegen id : 1] @@ -754,7 +754,7 @@ Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] (133) Filter [codegen id : 1] Input [4]: [d_week_seq#28, d_year#11, d_moy#86, d_dom#87] -Condition : (((((isnotnull(d_dom#87) AND isnotnull(d_moy#86)) AND isnotnull(d_year#11)) AND (d_year#11 = 1998)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) +Condition : (((((isnotnull(d_year#11) AND isnotnull(d_moy#86)) AND isnotnull(d_dom#87)) AND (d_year#11 = 1998)) AND (d_moy#86 = 12)) AND (d_dom#87 = 16)) (134) Project [codegen id : 1] Output [1]: [d_week_seq#28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index f1668ea399807..6e6950d4cb33a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -1,204 +1,204 @@ -TakeOrderedAndProject [channel,channel,i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,number_sales,number_sales,sales,sales] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id] - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #2 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #12 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #13 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [d_date_sk] #13 WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #13 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #1 + Exchange [i_brand_id,i_class_id,i_category_id] #1 WholeStageCodegen (25) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] BroadcastHashJoin [ss_item_sk,ss_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (11) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (10) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #4 + Exchange [brand_id,class_id,category_id] #4 WholeStageCodegen (9) - HashAggregate [brand_id,category_id,class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_category_id,i_class_id] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter BroadcastExchange #8 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #9 WholeStageCodegen (8) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #8 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (23) BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #2 InputAdapter BroadcastExchange #11 WholeStageCodegen (24) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #1 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #14 WholeStageCodegen (51) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #2 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #15 + Exchange [i_brand_id,i_class_id,i_category_id] #15 WholeStageCodegen (50) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] BroadcastHashJoin [ss_item_sk,ss_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] InputAdapter ReusedExchange [ss_item_sk] #2 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #10 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 InputAdapter BroadcastExchange #16 WholeStageCodegen (49) Project [d_date_sk] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] Subquery #3 WholeStageCodegen (1) Project [d_week_seq] - Filter [d_dom,d_moy,d_year] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_dom,d_moy,d_week_seq,d_year] + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index 0a76360d004ca..8c697ff080952 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -249,7 +249,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (7) ColumnarToRow [codegen id : 17] @@ -257,7 +257,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (8) Filter [codegen id : 17] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (9) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] @@ -308,7 +308,7 @@ Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, d_date_sk#10] Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_class_id)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] @@ -316,7 +316,7 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (21) Filter [codegen id : 4] Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) AND isnotnull(i_class_id#8)) +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (22) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index 7a78f19e5cc99..e4a9b46cf741d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,104 +1,104 @@ -TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] WholeStageCodegen (607) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #1 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 WholeStageCodegen (606) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (485) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #2 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #2 WholeStageCodegen (484) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (363) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #3 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #3 WholeStageCodegen (362) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #4 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #4 WholeStageCodegen (240) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (119) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #5 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #5 WholeStageCodegen (118) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (39) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #20 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #21 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter BroadcastExchange #22 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #22 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #6 + Exchange [i_brand_id,i_class_id,i_category_id] #6 WholeStageCodegen (38) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_item_sk,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] WholeStageCodegen (2) @@ -109,72 +109,72 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] WholeStageCodegen (18) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #8 WholeStageCodegen (17) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #9 WholeStageCodegen (16) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #10 + Exchange [brand_id,class_id,category_id] #10 WholeStageCodegen (15) - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - SortMergeJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (6) - Sort [brand_id,category_id,class_id] + Sort [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #11 + Exchange [brand_id,class_id,category_id] #11 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #12 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #13 WholeStageCodegen (4) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (10) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #14 + Exchange [i_brand_id,i_class_id,i_category_id] #14 WholeStageCodegen (9) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Project [cs_item_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter ReusedExchange [d_date_sk] #12 InputAdapter @@ -183,32 +183,32 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (14) - Sort [i_brand_id,i_category_id,i_class_id] + Sort [i_brand_id,i_class_id,i_category_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #16 + Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (13) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter ReusedExchange [d_date_sk] #12 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #17 WholeStageCodegen (19) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #18 SortMergeJoin [i_item_sk,ss_item_sk] @@ -220,23 +220,23 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] WholeStageCodegen (37) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #8 WholeStageCodegen (78) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #23 + Exchange [i_brand_id,i_class_id,i_category_id] #23 WholeStageCodegen (77) - HashAggregate [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] [count,count,isEmpty,isEmpty,sum,sum] - Project [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_list_price,cs_quantity] + Project [cs_item_sk,cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] @@ -248,7 +248,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] WholeStageCodegen (57) Sort [ss_item_sk] InputAdapter @@ -256,22 +256,22 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, InputAdapter ReusedExchange [d_date_sk] #17 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #18 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (117) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #25 + Exchange [i_brand_id,i_class_id,i_category_id] #25 WholeStageCodegen (116) - HashAggregate [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [ws_item_sk,ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter - SortMergeJoin [ss_item_sk,ws_item_sk] + SortMergeJoin [ws_item_sk,ss_item_sk] WholeStageCodegen (80) Sort [ws_item_sk] InputAdapter @@ -280,7 +280,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] WholeStageCodegen (96) Sort [ss_item_sk] InputAdapter @@ -288,140 +288,140 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, InputAdapter ReusedExchange [d_date_sk] #17 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #18 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #18 WholeStageCodegen (239) - HashAggregate [channel,i_brand_id,i_class_id,isEmpty,sum,sum] [i_category_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 WholeStageCodegen (238) - HashAggregate [channel,i_brand_id,i_class_id,number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #28 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 WholeStageCodegen (237) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (158) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (197) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #23 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (236) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #25 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 WholeStageCodegen (361) - HashAggregate [channel,i_brand_id,isEmpty,sum,sum] [i_category_id,i_class_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #29 WholeStageCodegen (360) - HashAggregate [channel,i_brand_id,number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #30 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #30 WholeStageCodegen (359) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (280) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (319) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #23 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (358) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #25 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 WholeStageCodegen (483) - HashAggregate [channel,isEmpty,sum,sum] [i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #31 WholeStageCodegen (482) - HashAggregate [channel,number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #32 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #32 WholeStageCodegen (481) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (402) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (441) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #23 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (480) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #25 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 WholeStageCodegen (605) - HashAggregate [isEmpty,sum,sum] [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #33 WholeStageCodegen (604) - HashAggregate [number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #34 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #34 WholeStageCodegen (603) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (524) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (563) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #23 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #23 WholeStageCodegen (602) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #25 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index e5afd3736c60c..c54ad0e36216d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -225,7 +225,7 @@ Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) Output [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_brand_id)] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 11] @@ -233,7 +233,7 @@ Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] (6) Filter [codegen id : 11] Input [4]: [i_item_sk#5, i_brand_id#6, i_class_id#7, i_category_id#8] -Condition : ((isnotnull(i_class_id#7) AND isnotnull(i_category_id#8)) AND isnotnull(i_brand_id#6)) +Condition : ((isnotnull(i_brand_id#6) AND isnotnull(i_class_id#7)) AND isnotnull(i_category_id#8)) (7) Scan parquet default.store_sales Output [2]: [ss_sold_date_sk#1, ss_item_sk#2] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index a329e40a70bbe..fc86da1801926 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,186 +1,186 @@ -TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] WholeStageCodegen (412) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #1 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 WholeStageCodegen (411) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (329) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #2 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #2 WholeStageCodegen (328) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (246) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #3 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #3 WholeStageCodegen (245) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (163) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] #4 + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #4 WholeStageCodegen (162) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #5 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #5 WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (26) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [average_sales,avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),count,sum] + HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(cast(quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2), true)),average_sales,sum,count] InputAdapter Exchange #17 WholeStageCodegen (7) - HashAggregate [list_price,quantity] [count,count,sum,sum] + HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union WholeStageCodegen (2) - Project [ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #18 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (4) - Project [cs_list_price,cs_quantity] + Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_quantity,cs_list_price] InputAdapter BroadcastExchange #19 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (6) - Project [ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [d_date_sk] #19 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #6 + Exchange [i_brand_id,i_class_id,i_category_id] #6 WholeStageCodegen (25) - HashAggregate [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_list_price,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] BroadcastHashJoin [ss_item_sk,ss_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_list_price,ss_quantity,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price] InputAdapter BroadcastExchange #7 WholeStageCodegen (11) Project [i_item_sk] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Filter [i_brand_id,i_category_id,i_class_id] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + Filter [i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (10) - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] - HashAggregate [brand_id,category_id,class_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] + HashAggregate [brand_id,class_id,category_id] InputAdapter - Exchange [brand_id,category_id,class_id] #9 + Exchange [brand_id,class_id,category_id] #9 WholeStageCodegen (9) - HashAggregate [brand_id,category_id,class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [brand_id,category_id,class_id,i_brand_id,i_category_id,i_class_id] - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [brand_id,class_id,category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk] InputAdapter BroadcastExchange #10 WholeStageCodegen (1) - Filter [i_brand_id,i_category_id,i_class_id,i_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #11 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #12 WholeStageCodegen (5) - Project [i_brand_id,i_category_id,i_class_id] + Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_category_id,i_class_id] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk] InputAdapter BroadcastExchange #13 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #11 InputAdapter BroadcastExchange #14 WholeStageCodegen (8) - Project [i_brand_id,i_category_id,i_class_id] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #11 InputAdapter @@ -190,198 +190,198 @@ TakeOrderedAndProject [channel,i_brand_id,i_category_id,i_class_id,number_sales, Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category_id,i_class_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #7 InputAdapter BroadcastExchange #16 WholeStageCodegen (24) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (52) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #20 + Exchange [i_brand_id,i_class_id,i_category_id] #20 WholeStageCodegen (51) - HashAggregate [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] [count,count,isEmpty,isEmpty,sum,sum] - Project [cs_list_price,cs_quantity,i_brand_id,i_category_id,i_class_id] + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_list_price,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id] + Project [cs_sold_date_sk,cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] BroadcastHashJoin [cs_item_sk,ss_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_list_price,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price] InputAdapter ReusedExchange [ss_item_sk] #7 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter ReusedExchange [d_date_sk] #16 WholeStageCodegen (78) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id] #21 + Exchange [i_brand_id,i_class_id,i_category_id] #21 WholeStageCodegen (77) - HashAggregate [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] [count,count,isEmpty,isEmpty,sum,sum] - Project [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,ws_list_price,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - BroadcastHashJoin [ss_item_sk,ws_item_sk] + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_list_price,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_list_price] InputAdapter ReusedExchange [ss_item_sk] #7 InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk] #15 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter ReusedExchange [d_date_sk] #16 WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,isEmpty,sum,sum] [i_category_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #22 WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #23 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #23 WholeStageCodegen (159) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (106) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (132) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #20 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (158) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #21 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 WholeStageCodegen (244) - HashAggregate [channel,i_brand_id,isEmpty,sum,sum] [i_category_id,i_class_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #24 WholeStageCodegen (243) - HashAggregate [channel,i_brand_id,number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #25 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #25 WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (189) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (215) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #20 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (241) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #21 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 WholeStageCodegen (327) - HashAggregate [channel,isEmpty,sum,sum] [i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #26 WholeStageCodegen (326) - HashAggregate [channel,number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #27 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #27 WholeStageCodegen (325) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (272) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (298) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #20 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (324) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #21 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 WholeStageCodegen (410) - HashAggregate [isEmpty,sum,sum] [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum,sum(number_salesL),sum(number_sales),sum(sum_sales),sum(sum_sales)] + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #28 WholeStageCodegen (409) - HashAggregate [number_sales,sum_sales] [isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,isEmpty,sum,sum] [isEmpty,number_sales,sum,sum,sum(number_salesL),sum(sales),sum_sales] + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_category_id,i_class_id] #29 + Exchange [channel,i_brand_id,i_class_id,i_category_id] #29 WholeStageCodegen (408) - HashAggregate [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] [isEmpty,isEmpty,sum,sum,sum,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union WholeStageCodegen (355) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #6 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #6 WholeStageCodegen (381) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #20 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #20 WholeStageCodegen (407) - Project [channel,i_brand_id,i_category_id,i_class_id,number_sales,sales] + Project [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #1 - HashAggregate [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] [channel,count,count(1),isEmpty,number_sales,sales,sum,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),channel,sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter - ReusedExchange [count,i_brand_id,i_category_id,i_class_id,isEmpty,sum] #21 + ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #21 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt index 9eb82fd1c6222..5514e335f1b51 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt @@ -1,42 +1,42 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,ca_state,i_item_id] +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union WholeStageCodegen (14) - HashAggregate [ca_country,ca_county,ca_state,count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [ca_country,ca_county,ca_state,i_item_id] #1 + Exchange [i_item_id,ca_country,ca_state,ca_county] #1 WholeStageCodegen (13) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,ca_state,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (5) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 WholeStageCodegen (4) - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [cd_demo_sk,cd_dep_count] - Filter [cd_demo_sk,cd_education_status,cd_gender] + Filter [cd_gender,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_count,cd_education_status,cd_gender] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -46,14 +46,14 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #6 WholeStageCodegen (11) - Project [c_birth_year,c_customer_sk,ca_country,ca_county,ca_state] + Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (8) @@ -61,20 +61,20 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Exchange [c_current_cdemo_sk] #7 WholeStageCodegen (7) - Project [c_birth_year,c_current_cdemo_sk,c_customer_sk,ca_country,ca_county,ca_state] + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_county,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter WholeStageCodegen (10) Sort [cd_demo_sk] @@ -86,25 +86,25 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Scan parquet default.customer_demographics [cd_demo_sk] WholeStageCodegen (28) - HashAggregate [ca_country,ca_state,count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),count,count,count,count,count,count,count,county,sum,sum,sum,sum,sum,sum,sum] + 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] InputAdapter - Exchange [ca_country,ca_state,i_item_id] #10 + Exchange [i_item_id,ca_country,ca_state] #10 WholeStageCodegen (27) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_state,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,ca_country,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (19) Sort [cs_bill_customer_sk] InputAdapter - ReusedExchange [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] #2 + ReusedExchange [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] #2 InputAdapter WholeStageCodegen (26) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #11 WholeStageCodegen (25) - Project [c_birth_year,c_customer_sk,ca_country,ca_state] + Project [c_customer_sk,c_birth_year,ca_state,ca_country] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (22) @@ -112,45 +112,45 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Exchange [c_current_cdemo_sk] #12 WholeStageCodegen (21) - Project [c_birth_year,c_current_cdemo_sk,c_customer_sk,ca_country,ca_state] + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_state,ca_country] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #13 WholeStageCodegen (20) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter WholeStageCodegen (24) Sort [cd_demo_sk] InputAdapter ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (42) - HashAggregate [ca_country,count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,count,count,count,count,count,count,count,county,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [i_item_id,ca_country,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),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [ca_country,i_item_id] #14 + Exchange [i_item_id,ca_country] #14 WholeStageCodegen (41) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,ca_country,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] - SortMergeJoin [c_customer_sk,cs_bill_customer_sk] + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (33) Sort [cs_bill_customer_sk] InputAdapter - ReusedExchange [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] #2 + ReusedExchange [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,i_item_id] #2 InputAdapter WholeStageCodegen (40) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #15 WholeStageCodegen (39) - Project [c_birth_year,c_customer_sk,ca_country] + Project [c_customer_sk,c_birth_year,ca_country] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (36) @@ -158,44 +158,44 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Exchange [c_current_cdemo_sk] #16 WholeStageCodegen (35) - Project [c_birth_year,c_current_cdemo_sk,c_customer_sk,ca_country] + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_country] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #17 WholeStageCodegen (34) Project [ca_address_sk,ca_country] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter WholeStageCodegen (38) Sort [cd_demo_sk] InputAdapter ReusedExchange [cd_demo_sk] #9 WholeStageCodegen (50) - HashAggregate [count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,count,count,count,count,count,count,count,county,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [i_item_id,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),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #18 WholeStageCodegen (49) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter @@ -203,23 +203,23 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter BroadcastExchange #19 WholeStageCodegen (47) - Project [c_birth_year,c_customer_sk] + Project [c_customer_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter BroadcastExchange #20 WholeStageCodegen (46) - Project [c_birth_year,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_birth_year] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #21 WholeStageCodegen (45) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] @@ -228,25 +228,25 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (58) - HashAggregate [count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,count,count,count,count,count,count,count,county,i_item_id,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [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),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange #22 WholeStageCodegen (57) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter ReusedExchange [cd_demo_sk,cd_dep_count] #3 InputAdapter @@ -259,4 +259,4 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter Scan parquet default.item [i_item_sk] InputAdapter - ReusedExchange [c_birth_year,c_customer_sk] #19 + ReusedExchange [c_customer_sk,c_birth_year] #19 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index bf528051893ed..49d50714a0940 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -1,43 +1,43 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,ca_state,i_item_id] +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union WholeStageCodegen (8) - HashAggregate [ca_country,ca_county,ca_state,count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [ca_country,ca_county,ca_state,i_item_id] #1 + Exchange [i_item_id,ca_country,ca_state,ca_county] #1 WholeStageCodegen (7) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,ca_state,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_year,ca_country,ca_county,ca_state,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk,cd_dep_count] - Filter [cd_demo_sk,cd_education_status,cd_gender] + Filter [cd_gender,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_count,cd_education_status,cd_gender] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] - Filter [c_birth_month,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_month,c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -48,15 +48,15 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_county,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -66,152 +66,152 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_county,c Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] WholeStageCodegen (16) - HashAggregate [ca_country,ca_state,count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),count,count,count,count,count,count,count,county,sum,sum,sum,sum,sum,sum,sum] + 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] InputAdapter - Exchange [ca_country,ca_state,i_item_id] #8 + Exchange [i_item_id,ca_country,ca_state] #8 WholeStageCodegen (15) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,ca_state,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,ca_country,ca_state,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,ca_country,ca_state,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_year,ca_country,ca_state,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter ReusedExchange [cd_demo_sk,cd_dep_count] #2 InputAdapter - ReusedExchange [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #3 InputAdapter ReusedExchange [cd_demo_sk] #4 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #7 WholeStageCodegen (24) - HashAggregate [ca_country,count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,count,count,count,count,count,count,count,county,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [i_item_id,ca_country,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),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [ca_country,i_item_id] #10 + Exchange [i_item_id,ca_country] #10 WholeStageCodegen (23) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,ca_country,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,ca_country,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,ca_country,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_year,ca_country,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter ReusedExchange [cd_demo_sk,cd_dep_count] #2 InputAdapter - ReusedExchange [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #3 InputAdapter ReusedExchange [cd_demo_sk] #4 InputAdapter BroadcastExchange #11 WholeStageCodegen (20) Project [ca_address_sk,ca_country] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #7 WholeStageCodegen (32) - HashAggregate [count,count,count,count,count,count,count,i_item_id,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,count,count,count,count,count,count,count,county,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [i_item_id,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),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #12 WholeStageCodegen (31) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7,i_item_id] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,i_item_id] + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter ReusedExchange [cd_demo_sk,cd_dep_count] #2 InputAdapter - ReusedExchange [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #3 InputAdapter ReusedExchange [cd_demo_sk] #4 InputAdapter BroadcastExchange #13 WholeStageCodegen (28) Project [ca_address_sk] - Filter [ca_address_sk,ca_state] + Filter [ca_state,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #7 WholeStageCodegen (40) - HashAggregate [count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,count,count,count,count,count,count,count,county,i_item_id,sum,sum,sum,sum,sum,sum,sum] + HashAggregate [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),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange #14 WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [count,count,count,count,count,count,count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_birth_year,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_birth_year,c_current_addr_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_addr_sk,c_birth_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,cd_dep_count,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - Project [cd_dep_count,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_sold_date_sk,cs_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_coupon_amt,cs_item_sk,cs_list_price,cs_net_profit,cs_quantity,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,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] InputAdapter ReusedExchange [cd_demo_sk,cd_dep_count] #2 InputAdapter - ReusedExchange [c_birth_year,c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #3 InputAdapter ReusedExchange [cd_demo_sk] #4 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt index b7d7a77003325..e66efff75180a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -8,12 +8,12 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(cs_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (6) - HashAggregate [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] [sum,sum] - Project [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) @@ -21,12 +21,12 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (2) - Project [cs_ext_sales_price,cs_item_sk] + Project [cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) @@ -34,7 +34,7 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -44,4 +44,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index b462752d01701..904e3ea00f358 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue,revenueratio] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -8,26 +8,26 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, InputAdapter Exchange [i_class] #1 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(cs_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #2 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 WholeStageCodegen (3) - HashAggregate [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] [sum,sum] - Project [cs_ext_sales_price,i_category,i_class,i_current_price,i_item_desc,i_item_id] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_category,i_class,i_current_price,i_item_desc,i_item_id] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -35,4 +35,4 @@ TakeOrderedAndProject [i_category,i_class,i_current_price,i_item_desc,i_item_id, Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt index ee408c2c2e004..d5f40d419175e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/simplified.txt @@ -1,16 +1,16 @@ -TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (9) - HashAggregate [count,i_brand,i_category,i_class,i_product_name,spark_grouping_id,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name,spark_grouping_id] #1 + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 WholeStageCodegen (8) - HashAggregate [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand,spark_grouping_id] [count,count,sum,sum] - Expand [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (6) - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] - SortMergeJoin [i_item_sk,inv_item_sk] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + SortMergeJoin [inv_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [inv_item_sk] @@ -18,7 +18,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Exchange [inv_item_sk] #2 WholeStageCodegen (2) Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [d_date_sk,inv_date_sk] + BroadcastHashJoin [inv_date_sk,d_date_sk] Filter [inv_date_sk,inv_item_sk] ColumnarToRow InputAdapter @@ -27,7 +27,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -40,7 +40,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] BroadcastExchange #5 WholeStageCodegen (7) ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 91c091e1d7c29..cdf9335b1c48b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -1,18 +1,18 @@ -TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] WholeStageCodegen (6) - HashAggregate [count,i_brand,i_category,i_class,i_product_name,spark_grouping_id,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name,spark_grouping_id] #1 + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 WholeStageCodegen (5) - HashAggregate [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand,spark_grouping_id] [count,count,sum,sum] - Expand [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (3) - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] - BroadcastHashJoin [i_item_sk,inv_item_sk] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [d_date_sk,inv_date_sk] + BroadcastHashJoin [inv_date_sk,d_date_sk] Filter [inv_date_sk,inv_item_sk] ColumnarToRow InputAdapter @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -31,7 +31,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] BroadcastExchange #4 WholeStageCodegen (4) ColumnarToRow diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt index efa57dd3395d0..c2fe3189b2d10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (8) - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(qoh),count,qoh,sum] - HashAggregate [i_brand,i_category,i_class,i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name] #1 + Exchange [i_product_name,i_brand,i_class,i_category] #1 WholeStageCodegen (7) - HashAggregate [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] [count,count,sum,sum] - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] - SortMergeJoin [i_item_sk,inv_item_sk] + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + SortMergeJoin [inv_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [inv_item_sk] @@ -17,13 +17,13 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Exchange [inv_item_sk] #2 WholeStageCodegen (3) Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [d_date_sk,inv_date_sk] + BroadcastHashJoin [inv_date_sk,d_date_sk] Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -35,7 +35,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] BroadcastExchange #4 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -48,40 +48,40 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (17) - HashAggregate [count,i_brand,i_class,i_product_name,sum] [avg(qoh),count,i_category,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] InputAdapter - Exchange [i_brand,i_class,i_product_name] #6 + Exchange [i_product_name,i_brand,i_class] #6 WholeStageCodegen (16) - HashAggregate [i_brand,i_class,i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (26) - HashAggregate [count,i_brand,i_product_name,sum] [avg(qoh),count,i_category,i_class,qoh,sum] + HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] InputAdapter - Exchange [i_brand,i_product_name] #7 + Exchange [i_product_name,i_brand] #7 WholeStageCodegen (25) - HashAggregate [i_brand,i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (35) - HashAggregate [count,i_product_name,sum] [avg(qoh),count,i_brand,i_category,i_class,qoh,sum] + HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] InputAdapter Exchange [i_product_name] #8 WholeStageCodegen (34) - HashAggregate [i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (44) - HashAggregate [count,sum] [avg(qoh),count,i_brand,i_category,i_class,i_product_name,qoh,sum] + HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] InputAdapter Exchange #9 WholeStageCodegen (43) - HashAggregate [qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 0b68b7090b0a2..616bfc89c0023 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,28 +1,28 @@ -TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (5) - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(qoh),count,qoh,sum] - HashAggregate [i_brand,i_category,i_class,i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name] #1 + Exchange [i_product_name,i_brand,i_class,i_category] #1 WholeStageCodegen (4) - HashAggregate [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] [count,count,sum,sum] - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand] + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [i_brand,i_category,i_class,i_product_name,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] - BroadcastHashJoin [d_date_sk,inv_date_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] Filter [inv_date_sk,inv_item_sk,inv_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -32,7 +32,7 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) @@ -41,38 +41,38 @@ TakeOrderedAndProject [i_brand,i_category,i_class,i_product_name,qoh] InputAdapter Scan parquet default.warehouse [w_warehouse_sk] WholeStageCodegen (11) - HashAggregate [count,i_brand,i_class,i_product_name,sum] [avg(qoh),count,i_category,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] InputAdapter - Exchange [i_brand,i_class,i_product_name] #5 + Exchange [i_product_name,i_brand,i_class] #5 WholeStageCodegen (10) - HashAggregate [i_brand,i_class,i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (17) - HashAggregate [count,i_brand,i_product_name,sum] [avg(qoh),count,i_category,i_class,qoh,sum] + HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] InputAdapter - Exchange [i_brand,i_product_name] #6 + Exchange [i_product_name,i_brand] #6 WholeStageCodegen (16) - HashAggregate [i_brand,i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (23) - HashAggregate [count,i_product_name,sum] [avg(qoh),count,i_brand,i_category,i_class,qoh,sum] + HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] InputAdapter Exchange [i_product_name] #7 WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 WholeStageCodegen (29) - HashAggregate [count,sum] [avg(qoh),count,i_brand,i_category,i_class,i_product_name,qoh,sum] + HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] InputAdapter Exchange #8 WholeStageCodegen (28) - HashAggregate [qoh] [count,count,sum,sum] - HashAggregate [count,i_brand,i_category,i_class,i_product_name,sum] [avg(cast(inv_quantity_on_hand as bigint)),count,qoh,sum] + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(cast(inv_quantity_on_hand as bigint)),qoh,sum,count] InputAdapter - ReusedExchange [count,i_brand,i_category,i_class,i_product_name,sum] #1 + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index ddcdbcd1237b2..9c1273c85439f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -101,7 +101,7 @@ Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country), IsNotNull(c_current_addr_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (13) ColumnarToRow @@ -109,7 +109,7 @@ Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name (14) Filter Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] -Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_birth_country#16)) AND isnotnull(c_current_addr_sk#13)) +Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) (15) BroadcastHashJoin [codegen id : 3] Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] @@ -176,11 +176,11 @@ Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name (29) Exchange Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#24, i_size#25, i_color#26, i_units#27, i_manager_id#28] -Arguments: hashpartitioning(cast(ss_item_sk#18 as bigint), cast(ss_ticket_number#21 as bigint), 5), true, [id=#30] +Arguments: hashpartitioning(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), true, [id=#30] (30) Sort [codegen id : 6] Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#24, i_size#25, i_color#26, i_units#27, i_manager_id#28] -Arguments: [cast(ss_item_sk#18 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST], false, 0 +Arguments: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 (31) Scan parquet default.store_returns Output [2]: [sr_item_sk#31, sr_ticket_number#32] @@ -198,15 +198,15 @@ Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) (34) Exchange Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_item_sk#31, sr_ticket_number#32, 5), true, [id=#33] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), true, [id=#33] (35) Sort [codegen id : 8] Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_item_sk#31 ASC NULLS FIRST, sr_ticket_number#32 ASC NULLS FIRST], false, 0 +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 (36) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_item_sk#18 as bigint), cast(ss_ticket_number#21 as bigint)] -Right keys [2]: [sr_item_sk#31, sr_ticket_number#32] +Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None (37) Project [codegen id : 9] @@ -369,7 +369,7 @@ Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country), IsNotNull(c_current_addr_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (60) ColumnarToRow @@ -377,7 +377,7 @@ Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name (61) Filter Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] -Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_birth_country#16)) AND isnotnull(c_current_addr_sk#13)) +Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) (62) BroadcastHashJoin [codegen id : 3] Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] @@ -456,11 +456,11 @@ Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name (79) Exchange Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#24, i_size#25, i_color#26, i_units#27, i_manager_id#28] -Arguments: hashpartitioning(cast(ss_item_sk#18 as bigint), cast(ss_ticket_number#21 as bigint), 5), true, [id=#55] +Arguments: hashpartitioning(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), true, [id=#55] (80) Sort [codegen id : 9] Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#24, i_size#25, i_color#26, i_units#27, i_manager_id#28] -Arguments: [cast(ss_item_sk#18 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST], false, 0 +Arguments: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 (81) Scan parquet default.store_returns Output [2]: [sr_item_sk#31, sr_ticket_number#32] @@ -478,15 +478,15 @@ Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) (84) Exchange Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_item_sk#31, sr_ticket_number#32, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), true, [id=#56] (85) Sort [codegen id : 11] Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_item_sk#31 ASC NULLS FIRST, sr_ticket_number#32 ASC NULLS FIRST], false, 0 +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 (86) SortMergeJoin [codegen id : 12] -Left keys [2]: [cast(ss_item_sk#18 as bigint), cast(ss_ticket_number#21 as bigint)] -Right keys [2]: [sr_item_sk#31, sr_ticket_number#32] +Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None (87) Project [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt index 9adad49cb8b1a..7b1c03ef15b15 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/simplified.txt @@ -1,70 +1,70 @@ WholeStageCodegen (12) - Sort [c_first_name,c_last_name,s_store_name] + Sort [c_last_name,c_first_name,s_store_name] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #1 + Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (11) - Project [c_first_name,c_last_name,paid,s_store_name] + Project [c_last_name,c_first_name,s_store_name,paid] Filter [sum(netpaid)] Subquery #1 WholeStageCodegen (14) - HashAggregate [count,sum] [(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),avg(netpaid),count,sum] + HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] InputAdapter Exchange #10 WholeStageCodegen (13) - HashAggregate [netpaid] [count,count,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #11 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 WholeStageCodegen (12) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (9) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #12 + Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (8) - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - SortMergeJoin [i_item_sk,ss_item_sk] + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (5) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #13 WholeStageCodegen (4) - Project [c_first_name,c_last_name,ca_state,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,s_store_sk,ss_customer_sk,ss_store_sk] + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] + BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] InputAdapter BroadcastExchange #14 WholeStageCodegen (3) - Project [c_customer_sk,c_first_name,c_last_name,ca_state,s_state,s_store_name,s_store_sk] - BroadcastHashJoin [c_birth_country,c_current_addr_sk,ca_address_sk,ca_country] + Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] + BroadcastHashJoin [ca_address_sk,ca_country,c_current_addr_sk,c_birth_country] InputAdapter BroadcastExchange #15 WholeStageCodegen (2) - Project [ca_address_sk,ca_country,ca_state,s_state,s_store_name,s_store_sk] - BroadcastHashJoin [ca_zip,s_zip] + Project [s_store_sk,s_store_name,s_state,ca_address_sk,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] InputAdapter BroadcastExchange #16 WholeStageCodegen (1) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] Filter [ca_address_sk,ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state,ca_zip] - Filter [c_birth_country,c_current_addr_sk,c_customer_sk] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter WholeStageCodegen (7) Sort [i_item_sk] @@ -74,83 +74,83 @@ WholeStageCodegen (12) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (11) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #18 + Exchange [sr_ticket_number,sr_item_sk] #18 WholeStageCodegen (10) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] - HashAggregate [c_first_name,c_last_name,isEmpty,s_store_name,sum] [isEmpty,paid,sum,sum(netpaid),sum(netpaid)] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #2 + Exchange [c_last_name,c_first_name,s_store_name] #2 WholeStageCodegen (10) - HashAggregate [c_first_name,c_last_name,netpaid,s_store_name] [isEmpty,isEmpty,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #3 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 WholeStageCodegen (9) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (6) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #4 + Exchange [ss_ticket_number,ss_item_sk] #4 WholeStageCodegen (5) - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [c_first_name,c_last_name,ca_state,s_state,s_store_name,ss_item_sk,ss_net_paid,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,s_store_sk,ss_customer_sk,ss_store_sk] + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] + BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) - Project [c_customer_sk,c_first_name,c_last_name,ca_state,s_state,s_store_name,s_store_sk] - BroadcastHashJoin [c_birth_country,c_current_addr_sk,ca_address_sk,ca_country] + Project [s_store_sk,s_store_name,s_state,ca_state,c_customer_sk,c_first_name,c_last_name] + BroadcastHashJoin [ca_address_sk,ca_country,c_current_addr_sk,c_birth_country] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) - Project [ca_address_sk,ca_country,ca_state,s_state,s_store_name,s_store_sk] - BroadcastHashJoin [ca_zip,s_zip] + Project [s_store_sk,s_store_name,s_state,ca_address_sk,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] InputAdapter BroadcastExchange #7 WholeStageCodegen (1) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] Filter [ca_address_sk,ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state,ca_zip] - Filter [c_birth_country,c_current_addr_sk,c_customer_sk] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) Filter [i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (8) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #9 + Exchange [sr_ticket_number,sr_item_sk] #9 WholeStageCodegen (7) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 870a1c24979c6..d4a429614c992 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -148,7 +148,7 @@ Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_s Output [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25, c_birth_country#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country), IsNotNull(c_current_addr_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -156,7 +156,7 @@ Input [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name (25) Filter [codegen id : 4] Input [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25, c_birth_country#26] -Condition : ((isnotnull(c_customer_sk#22) AND isnotnull(c_birth_country#26)) AND isnotnull(c_current_addr_sk#23)) +Condition : ((isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) AND isnotnull(c_birth_country#26)) (26) BroadcastExchange Input [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25, c_birth_country#26] @@ -175,7 +175,7 @@ Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip Output [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip), IsNotNull(ca_country)] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -183,7 +183,7 @@ Input [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] (31) Filter [codegen id : 5] Input [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] -Condition : ((isnotnull(ca_address_sk#28) AND isnotnull(ca_zip#30)) AND isnotnull(ca_country#31)) +Condition : ((isnotnull(ca_address_sk#28) AND isnotnull(ca_country#31)) AND isnotnull(ca_zip#30)) (32) BroadcastExchange Input [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] @@ -398,7 +398,7 @@ Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_s Output [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25, c_birth_country#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country), IsNotNull(c_current_addr_sk)] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (68) ColumnarToRow [codegen id : 4] @@ -406,7 +406,7 @@ Input [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name (69) Filter [codegen id : 4] Input [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25, c_birth_country#26] -Condition : ((isnotnull(c_customer_sk#22) AND isnotnull(c_birth_country#26)) AND isnotnull(c_current_addr_sk#23)) +Condition : ((isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) AND isnotnull(c_birth_country#26)) (70) BroadcastExchange Input [5]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25, c_birth_country#26] @@ -425,7 +425,7 @@ Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#10, s_state#12, s_zip Output [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip), IsNotNull(ca_country)] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (74) ColumnarToRow [codegen id : 5] @@ -433,7 +433,7 @@ Input [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] (75) Filter [codegen id : 5] Input [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] -Condition : ((isnotnull(ca_address_sk#28) AND isnotnull(ca_zip#30)) AND isnotnull(ca_country#31)) +Condition : ((isnotnull(ca_address_sk#28) AND isnotnull(ca_country#31)) AND isnotnull(ca_zip#30)) (76) BroadcastExchange Input [4]: [ca_address_sk#28, ca_state#29, ca_zip#30, ca_country#31] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index ee0ecc649b7dd..2baafcbe13da0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -1,129 +1,129 @@ WholeStageCodegen (9) - Sort [c_first_name,c_last_name,s_store_name] + Sort [c_last_name,c_first_name,s_store_name] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #1 + Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (8) - Project [c_first_name,c_last_name,paid,s_store_name] + Project [c_last_name,c_first_name,s_store_name,paid] Filter [sum(netpaid)] Subquery #1 WholeStageCodegen (8) - HashAggregate [count,sum] [(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),avg(netpaid),count,sum] + HashAggregate [sum,count] [avg(netpaid),(CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6))),sum,count] InputAdapter Exchange #9 WholeStageCodegen (7) - HashAggregate [netpaid] [count,count,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #10 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - BroadcastHashJoin [c_birth_country,c_current_addr_sk,ca_address_sk,ca_country,ca_zip,s_zip] - Project [c_birth_country,c_current_addr_sk,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #11 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #12 WholeStageCodegen (2) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #13 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - Filter [c_birth_country,c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #15 WholeStageCodegen (5) Filter [ca_address_sk,ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state,ca_zip] - HashAggregate [c_first_name,c_last_name,isEmpty,s_store_name,sum] [isEmpty,paid,sum,sum(netpaid),sum(netpaid)] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] InputAdapter - Exchange [c_first_name,c_last_name,s_store_name] #2 + Exchange [c_last_name,c_first_name,s_store_name] #2 WholeStageCodegen (7) - HashAggregate [c_first_name,c_last_name,netpaid,s_store_name] [isEmpty,isEmpty,sum,sum] - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,sum] [netpaid,sum,sum(UnscaledValue(ss_net_paid))] + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] InputAdapter - Exchange [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name] #3 + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 WholeStageCodegen (6) - HashAggregate [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] [sum,sum] - Project [c_first_name,c_last_name,ca_state,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,ss_net_paid] - BroadcastHashJoin [c_birth_country,c_current_addr_sk,ca_address_sk,ca_country,ca_zip,s_zip] - Project [c_birth_country,c_current_addr_sk,c_first_name,c_last_name,i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_net_paid] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [i_color,i_current_price,i_manager_id,i_size,i_units,s_state,s_store_name,s_zip,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,s_store_name,s_zip,ss_customer_sk,ss_item_sk,ss_net_paid] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_store_sk,ss_ticket_number] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_net_paid,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Project [s_state,s_store_name,s_store_sk,s_zip] + Project [s_store_sk,s_store_name,s_state,s_zip] Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_market_id,s_state,s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Filter [i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_manager_id,i_size,i_units] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (4) - Filter [c_birth_country,c_current_addr_sk,c_customer_sk] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.customer [c_birth_country,c_current_addr_sk,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 WholeStageCodegen (5) Filter [ca_address_sk,ca_country,ca_zip] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_country,ca_state,ca_zip] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt index fc684d188d714..86722de954ff3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt @@ -96,7 +96,7 @@ Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND is Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_education_status), IsNotNull(cd_gender), IsNotNull(cd_marital_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -104,7 +104,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_education_status#12) AND isnotnull(cd_gender#10)) AND isnotnull(cd_marital_status#11)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = W)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = W)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/simplified.txt index cc47c4b7bc332..61aaa22964cb7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,s_state,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,sum,sum,sum,sum] + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state] #1 WholeStageCodegen (5) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id,s_state] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -40,32 +40,32 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] WholeStageCodegen (12) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,s_state,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),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] InputAdapter Exchange [i_item_id] #6 WholeStageCodegen (11) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [cd_demo_sk] #2 InputAdapter @@ -75,29 +75,29 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [count,count,count,count,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,i_item_id,s_state,sum,sum,sum,sum] + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange #8 WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [cd_demo_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index 8ff35202e0f18..75e07ce8cd680 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -96,7 +96,7 @@ Condition : (((isnotnull(ss_cdemo_sk#3) AND isnotnull(ss_sold_date_sk#1)) AND is Output [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_education_status), IsNotNull(cd_marital_status), IsNotNull(cd_gender), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -104,7 +104,7 @@ Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_statu (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#9, cd_gender#10, cd_marital_status#11, cd_education_status#12] -Condition : ((((((isnotnull(cd_education_status#12) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_gender#10)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = W)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) +Condition : ((((((isnotnull(cd_gender#10) AND isnotnull(cd_marital_status#11)) AND isnotnull(cd_education_status#12)) AND (cd_gender#10 = F)) AND (cd_marital_status#11 = W)) AND (cd_education_status#12 = Primary)) AND isnotnull(cd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 7bfdbae974b51..995ed4b432be6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -1,36 +1,36 @@ -TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union WholeStageCodegen (6) - HashAggregate [count,count,count,count,i_item_id,s_state,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,sum,sum,sum,sum] + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state] #1 WholeStageCodegen (5) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id,s_state] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,s_state,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_state,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Filter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_education_status,cd_gender,cd_marital_status] + Scan parquet default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -40,32 +40,32 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id] WholeStageCodegen (12) - HashAggregate [count,count,count,count,i_item_id,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,s_state,sum,sum,sum,sum] + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),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] InputAdapter Exchange [i_item_id] #6 WholeStageCodegen (11) - HashAggregate [agg1,agg2,agg3,agg4,i_item_id] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [i_item_id,ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [cd_demo_sk] #2 InputAdapter @@ -77,27 +77,27 @@ TakeOrderedAndProject [agg1,agg2,agg3,agg4,g_state,i_item_id,s_state] Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter - ReusedExchange [i_item_id,i_item_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) - HashAggregate [count,count,count,count,sum,sum,sum,sum] [agg1,agg2,agg3,agg4,avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),avg(cast(agg1 as bigint)),count,count,count,count,g_state,i_item_id,s_state,sum,sum,sum,sum] + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(cast(agg1 as bigint)),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange #8 WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [count,count,count,count,count,count,count,count,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_coupon_amt,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Filter [ss_cdemo_sk,ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_cdemo_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] InputAdapter ReusedExchange [cd_demo_sk] #2 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt index cd212364ff4f7..451659e2c617c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/simplified.txt @@ -1,10 +1,10 @@ WholeStageCodegen (10) - Sort [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,ss_ticket_number] + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] InputAdapter - Exchange [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,ss_ticket_number] #1 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 WholeStageCodegen (9) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] @@ -12,29 +12,29 @@ WholeStageCodegen (10) Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #3 + Exchange [ss_ticket_number,ss_customer_sk] #3 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -42,15 +42,15 @@ WholeStageCodegen (10) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter WholeStageCodegen (8) Sort [c_customer_sk] @@ -60,4 +60,4 @@ WholeStageCodegen (10) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index 0bdcd83bf2968..8aa32fed5a176 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -1,34 +1,34 @@ WholeStageCodegen (7) - Sort [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,ss_ticket_number] + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] InputAdapter - Exchange [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,ss_ticket_number] #1 + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 WholeStageCodegen (6) - Project [c_first_name,c_last_name,c_preferred_cust_flag,c_salutation,cnt,ss_ticket_number] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] - HashAggregate [count,ss_customer_sk,ss_ticket_number] [cnt,count,count(1)] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter - Exchange [ss_customer_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_customer_sk] #2 WholeStageCodegen (4) - HashAggregate [ss_customer_sk,ss_ticket_number] [count,count] + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [s_store_sk,ss_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_hdemo_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_dom,d_year] + Filter [d_dom,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_dom,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_dom] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) @@ -36,19 +36,19 @@ WholeStageCodegen (7) Filter [s_county,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_store_sk] + Scan parquet default.store [s_store_sk,s_county] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Filter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk,hd_dep_count,hd_vehicle_count] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) Filter [c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_preferred_cust_flag,c_salutation] + Scan parquet default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt index 3e21f6dae18d6..b1e750592d99e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/simplified.txt @@ -1,11 +1,11 @@ -TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,cnt1,cnt2,cnt3,max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_employed_count),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] WholeStageCodegen (21) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,count,count,count,count,max,max,max,sum,sum,sum,sum,sum,sum] [avg(cast(cd_dep_college_count as bigint)),avg(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),cnt1,cnt2,cnt3,count,count,count,count,count(1),max,max,max,max(cd_dep_college_count),max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_count),max(cd_dep_employed_count),max(cd_dep_employed_count),sum,sum,sum,sum,sum,sum,sum(cast(cd_dep_college_count as bigint)),sum(cast(cd_dep_count as bigint)),sum(cast(cd_dep_employed_count as bigint)),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] + HashAggregate [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] [count(1),avg(cast(cd_dep_count as bigint)),max(cd_dep_count),sum(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),max(cd_dep_employed_count),sum(cast(cd_dep_employed_count as bigint)),avg(cast(cd_dep_college_count as bigint)),max(cd_dep_college_count),sum(cast(cd_dep_college_count as bigint)),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter - Exchange [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] #1 + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (20) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] [count,count,count,count,count,count,count,count,max,max,max,max,max,max,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (17) @@ -21,7 +21,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em InputAdapter Exchange [c_current_addr_sk] #3 WholeStageCodegen (12) - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] InputAdapter SortMergeJoin [c_customer_sk,cs_ship_customer_sk] @@ -35,37 +35,37 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #5 WholeStageCodegen (4) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_year,d_qoy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] WholeStageCodegen (8) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #7 WholeStageCodegen (7) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (11) @@ -78,7 +78,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -100,4 +100,4 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index 2da6d615fe17d..4f6a2929c32ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,cnt1,cnt2,cnt3,max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_employed_count),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] WholeStageCodegen (10) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,count,count,count,count,max,max,max,sum,sum,sum,sum,sum,sum] [avg(cast(cd_dep_college_count as bigint)),avg(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),cnt1,cnt2,cnt3,count,count,count,count,count(1),max,max,max,max(cd_dep_college_count),max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_count),max(cd_dep_employed_count),max(cd_dep_employed_count),sum,sum,sum,sum,sum,sum,sum(cast(cd_dep_college_count as bigint)),sum(cast(cd_dep_count as bigint)),sum(cast(cd_dep_employed_count as bigint)),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] + HashAggregate [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] [count(1),avg(cast(cd_dep_count as bigint)),max(cd_dep_count),sum(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),max(cd_dep_employed_count),sum(cast(cd_dep_employed_count as bigint)),avg(cast(cd_dep_college_count as bigint)),max(cd_dep_college_count),sum(cast(cd_dep_college_count as bigint)),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter - Exchange [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] #1 + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (9) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] [count,count,count,count,count,count,count,count,max,max,max,max,max,max,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk,ca_state] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] @@ -17,33 +17,33 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_year,d_qoy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -54,7 +54,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -70,4 +70,4 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt index 2840f275ff2b7..b38eeeb527eb4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/simplified.txt @@ -1,11 +1,11 @@ -TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,cnt1,cnt2,cnt3,max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_employed_count),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] WholeStageCodegen (20) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,count,count,count,count,max,max,max,sum,sum,sum,sum,sum,sum] [avg(cast(cd_dep_college_count as bigint)),avg(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),cnt1,cnt2,cnt3,count,count,count,count,count(1),max,max,max,max(cd_dep_college_count),max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_count),max(cd_dep_employed_count),max(cd_dep_employed_count),sum,sum,sum,sum,sum,sum,sum(cast(cd_dep_college_count as bigint)),sum(cast(cd_dep_count as bigint)),sum(cast(cd_dep_employed_count as bigint)),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] + HashAggregate [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] [count(1),avg(cast(cd_dep_count as bigint)),max(cd_dep_count),sum(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),max(cd_dep_employed_count),sum(cast(cd_dep_employed_count as bigint)),avg(cast(cd_dep_college_count as bigint)),max(cd_dep_college_count),sum(cast(cd_dep_college_count as bigint)),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter - Exchange [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] #1 + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (19) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] [count,count,count,count,count,count,count,count,max,max,max,max,max,max,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (16) @@ -21,7 +21,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em InputAdapter Exchange [c_current_addr_sk] #3 WholeStageCodegen (11) - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] InputAdapter SortMergeJoin [c_customer_sk,customsk] SortMergeJoin [c_customer_sk,ss_customer_sk] @@ -33,26 +33,26 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] WholeStageCodegen (5) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #5 WholeStageCodegen (4) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_year,d_qoy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] WholeStageCodegen (10) Sort [customsk] InputAdapter @@ -60,11 +60,11 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Union WholeStageCodegen (7) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #6 WholeStageCodegen (9) @@ -73,7 +73,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -95,4 +95,4 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index 054296501acf2..85198feb8e903 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -1,48 +1,48 @@ -TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,cnt1,cnt2,cnt3,max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_employed_count),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] WholeStageCodegen (10) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status,count,count,count,count,max,max,max,sum,sum,sum,sum,sum,sum] [avg(cast(cd_dep_college_count as bigint)),avg(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_employed_count),cnt1,cnt2,cnt3,count,count,count,count,count(1),max,max,max,max(cd_dep_college_count),max(cd_dep_college_count),max(cd_dep_count),max(cd_dep_count),max(cd_dep_employed_count),max(cd_dep_employed_count),sum,sum,sum,sum,sum,sum,sum(cast(cd_dep_college_count as bigint)),sum(cast(cd_dep_count as bigint)),sum(cast(cd_dep_employed_count as bigint)),sum(cd_dep_college_count),sum(cd_dep_count),sum(cd_dep_employed_count)] + HashAggregate [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] [count(1),avg(cast(cd_dep_count as bigint)),max(cd_dep_count),sum(cast(cd_dep_count as bigint)),avg(cast(cd_dep_employed_count as bigint)),max(cd_dep_employed_count),sum(cast(cd_dep_employed_count as bigint)),avg(cast(cd_dep_college_count as bigint)),max(cd_dep_college_count),sum(cast(cd_dep_college_count as bigint)),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter - Exchange [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] #1 + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (9) - HashAggregate [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] [count,count,count,count,count,count,count,count,max,max,max,max,max,max,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ca_state,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] Project [c_current_cdemo_sk,ca_state] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,c_current_cdemo_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] Filter [c_current_addr_sk,c_current_cdemo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (2) Project [ss_customer_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_qoy,d_year] + Filter [d_year,d_qoy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #4 Union WholeStageCodegen (4) Project [ws_bill_customer_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 WholeStageCodegen (6) @@ -51,7 +51,7 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter @@ -67,4 +67,4 @@ TakeOrderedAndProject [avg(cd_dep_college_count),avg(cd_dep_count),avg(cd_dep_em Filter [cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_dep_college_count,cd_dep_count,cd_dep_employed_count,cd_gender,cd_marital_status] + Scan parquet 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/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/simplified.txt index 2338ff8061e19..aa85d4870683d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/simplified.txt @@ -1,49 +1,49 @@ -TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] WholeStageCodegen (23) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w0,gross_margin,lochierarchy] + Window [gross_margin,lochierarchy,_w0] WholeStageCodegen (22) - Sort [_w0,gross_margin,lochierarchy] + Sort [lochierarchy,_w0,gross_margin] InputAdapter - Exchange [_w0,lochierarchy] #1 + Exchange [lochierarchy,_w0] #1 WholeStageCodegen (21) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] [_w0] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] InputAdapter - Exchange [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] #2 + Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 WholeStageCodegen (20) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union WholeStageCodegen (13) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter - Exchange [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] #3 + Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #3 WholeStageCodegen (12) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [gross_margin,lochierarchy,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),t_category,t_class] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] InputAdapter Exchange [i_category,i_class] #4 WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [i_category,i_class,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -54,29 +54,29 @@ TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_ Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] WholeStageCodegen (11) - HashAggregate [i_category,isEmpty,isEmpty,sum,sum] [gross_margin,i_class,isEmpty,isEmpty,lochierarchy,sum,sum,sum(ss_ext_sales_price),sum(ss_net_profit),t_category,t_class] + 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] InputAdapter Exchange [i_category] #8 WholeStageCodegen (10) - HashAggregate [i_category,ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [i_category,i_class,sum,sum] [ss_ext_sales_price,ss_net_profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum,sum] #4 WholeStageCodegen (19) - HashAggregate [isEmpty,isEmpty,sum,sum] [gross_margin,i_category,i_class,isEmpty,isEmpty,lochierarchy,sum,sum,sum(ss_ext_sales_price),sum(ss_net_profit),t_category,t_class] + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #9 WholeStageCodegen (18) - HashAggregate [ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [i_category,i_class,sum,sum] [ss_ext_sales_price,ss_net_profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum,sum] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 9cc1e03f3ec4f..a72781e1da0ed 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,49 +1,49 @@ -TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] WholeStageCodegen (23) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w0,gross_margin,lochierarchy] + Window [gross_margin,lochierarchy,_w0] WholeStageCodegen (22) - Sort [_w0,gross_margin,lochierarchy] + Sort [lochierarchy,_w0,gross_margin] InputAdapter - Exchange [_w0,lochierarchy] #1 + Exchange [lochierarchy,_w0] #1 WholeStageCodegen (21) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] [_w0] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] InputAdapter - Exchange [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] #2 + Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 WholeStageCodegen (20) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union WholeStageCodegen (13) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter - Exchange [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] #3 + Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #3 WholeStageCodegen (12) - HashAggregate [gross_margin,i_category,i_class,lochierarchy,t_category,t_class] + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [gross_margin,lochierarchy,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),t_category,t_class] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] InputAdapter Exchange [i_category,i_class] #4 WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [i_category,i_class,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [i_category,i_class,ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -53,7 +53,7 @@ TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_ Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] InputAdapter BroadcastExchange #7 WholeStageCodegen (3) @@ -61,22 +61,22 @@ TakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_ Filter [s_state,s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] WholeStageCodegen (11) - HashAggregate [i_category,isEmpty,isEmpty,sum,sum] [gross_margin,i_class,isEmpty,isEmpty,lochierarchy,sum,sum,sum(ss_ext_sales_price),sum(ss_net_profit),t_category,t_class] + 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] InputAdapter Exchange [i_category] #8 WholeStageCodegen (10) - HashAggregate [i_category,ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [i_category,i_class,sum,sum] [ss_ext_sales_price,ss_net_profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum,sum] #4 WholeStageCodegen (19) - HashAggregate [isEmpty,isEmpty,sum,sum] [gross_margin,i_category,i_class,isEmpty,isEmpty,lochierarchy,sum,sum,sum(ss_ext_sales_price),sum(ss_net_profit),t_category,t_class] + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #9 WholeStageCodegen (18) - HashAggregate [ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum] - HashAggregate [i_category,i_class,sum,sum] [ss_ext_sales_price,ss_net_profit,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum,sum] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index f078ca8cd68d3..b09c0073bc795 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -104,7 +104,7 @@ Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_store_sk#3, ss_sales_price#4, d_ Output [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_company_name), IsNotNull(s_store_name)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -112,7 +112,7 @@ Input [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] (12) Filter [codegen id : 2] Input [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] -Condition : ((isnotnull(s_store_sk#9) AND isnotnull(s_company_name#11)) AND isnotnull(s_store_name#10)) +Condition : ((isnotnull(s_store_sk#9) AND isnotnull(s_store_name#10)) AND isnotnull(s_company_name#11)) (13) BroadcastExchange Input [3]: [s_store_sk#9, s_store_name#10, s_company_name#11] @@ -139,7 +139,7 @@ Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 Output [3]: [i_item_sk#14, i_brand#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand), IsNotNull(i_category)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] ReadSchema: struct (19) ColumnarToRow [codegen id : 5] @@ -147,7 +147,7 @@ Input [3]: [i_item_sk#14, i_brand#15, i_category#16] (20) Filter [codegen id : 5] Input [3]: [i_item_sk#14, i_brand#15, i_category#16] -Condition : ((isnotnull(i_item_sk#14) AND isnotnull(i_brand#15)) AND isnotnull(i_category#16)) +Condition : ((isnotnull(i_item_sk#14) AND isnotnull(i_category#16)) AND isnotnull(i_brand#15)) (21) Exchange Input [3]: [i_item_sk#14, i_brand#15, i_category#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt index e91829b2812f8..015e1c88331f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/simplified.txt @@ -1,107 +1,107 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_category,nsum,psum,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (37) - Project [avg_monthly_sales,d_moy,d_year,i_category,sum_sales,sum_sales,sum_sales] - SortMergeJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (25) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,rn,s_company_name,s_store_name,sum_sales,sum_sales] - SortMergeJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + SortMergeJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter WholeStageCodegen (13) - Sort [i_brand,i_category,rn,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_brand,i_category,rn,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #1 WholeStageCodegen (12) - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (11) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #2 + Exchange [i_category,i_brand,s_store_name,s_company_name] #2 WholeStageCodegen (10) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] WholeStageCodegen (9) - Sort [d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year] InputAdapter - Exchange [d_year,i_brand,i_category,s_company_name,s_store_name] #3 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year] #3 WholeStageCodegen (8) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] #4 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #4 WholeStageCodegen (7) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 WholeStageCodegen (3) - Project [d_moy,d_year,s_company_name,s_store_name,ss_item_sk,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_year,ss_item_sk,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) - Filter [s_company_name,s_store_name,s_store_sk] + Filter [s_store_sk,s_store_name,s_company_name] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #8 WholeStageCodegen (5) - Filter [i_brand,i_category,i_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (24) - Sort [i_brand,i_category,rn,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_brand,i_category,rn,s_company_name,s_store_name] #9 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #9 WholeStageCodegen (23) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (22) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #10 + Exchange [i_category,i_brand,s_store_name,s_company_name] #10 WholeStageCodegen (21) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] #4 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #4 InputAdapter WholeStageCodegen (36) - Sort [i_brand,i_category,rn,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,rn] InputAdapter - Exchange [i_brand,i_category,rn,s_company_name,s_store_name] #11 + Exchange [i_category,i_brand,s_store_name,s_company_name,rn] #11 WholeStageCodegen (35) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (34) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] #10 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 7e007a9d138c7..f06dc36166d2e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -195,7 +195,7 @@ Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#10, d_moy#11, sum_sales#21, avg_monthly_sales#24, rn#26] -Condition : (((((isnotnull(avg_monthly_sales#24) AND isnotnull(d_year#10)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#24 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#26)) +Condition : (((((isnotnull(d_year#10) AND isnotnull(avg_monthly_sales#24)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#24 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#26)) (33) ReusedExchange [Reuses operator id: 23] Output [7]: [i_category#27, i_brand#28, s_store_name#29, s_company_name#30, d_year#31, d_moy#32, sum#33] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index 98c0f46ab2b71..df07dd8098e79 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,84 +1,84 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_category,nsum,psum,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] WholeStageCodegen (23) - Project [avg_monthly_sales,d_moy,d_year,i_category,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,rn,s_company_name,s_store_name,sum_sales,sum_sales] - BroadcastHashJoin [i_brand,i_brand,i_category,i_category,rn,rn,s_company_name,s_company_name,s_store_name,s_store_name] - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (8) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #1 + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 WholeStageCodegen (7) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] WholeStageCodegen (6) - Sort [d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year] InputAdapter - Exchange [d_year,i_brand,i_category,s_company_name,s_store_name] #2 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year] #2 WholeStageCodegen (5) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [_w0,sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] #3 + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #3 WholeStageCodegen (4) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] [sum,sum] - Project [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_year,i_brand,i_category,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand,i_category,ss_sales_price,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_sold_date_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_brand,i_category,i_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - Filter [s_company_name,s_store_name,s_store_sk] + Filter [s_store_sk,s_store_name,s_company_name] ColumnarToRow InputAdapter - Scan parquet default.store [s_company_name,s_store_name,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #7 WholeStageCodegen (15) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (14) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_brand,i_category,s_company_name,s_store_name] #8 + Exchange [i_category,i_brand,s_store_name,s_company_name] #8 WholeStageCodegen (13) - HashAggregate [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] [sum,sum(UnscaledValue(ss_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum] #3 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #3 InputAdapter BroadcastExchange #9 WholeStageCodegen (22) - Project [i_brand,i_category,rn,s_company_name,s_store_name,sum_sales] + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] Filter [rn] InputAdapter - Window [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] WholeStageCodegen (21) - Sort [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name] + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [d_moy,d_year,i_brand,i_category,s_company_name,s_store_name,sum_sales] #8 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index 31810073746e7..22479caee2de1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -149,7 +149,7 @@ Arguments: [cast(ws_order_number#3 as bigint) ASC NULLS FIRST, cast(ws_item_sk#2 Output [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_amt#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] @@ -157,7 +157,7 @@ Input [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_ (16) Filter [codegen id : 4] Input [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_amt#15] -Condition : (((isnotnull(wr_return_amt#15) AND (wr_return_amt#15 > 10000.00)) AND isnotnull(wr_item_sk#12)) AND isnotnull(wr_order_number#13)) +Condition : (((isnotnull(wr_return_amt#15) AND (wr_return_amt#15 > 10000.00)) AND isnotnull(wr_order_number#13)) AND isnotnull(wr_item_sk#12)) (17) Exchange Input [4]: [wr_item_sk#12, wr_order_number#13, wr_return_quantity#14, wr_return_amt#15] @@ -226,7 +226,7 @@ Input [5]: [item#34, return_ratio#35, currency_ratio#36, return_rank#38, currenc Output [6]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_net_profit#46] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_net_paid), IsNotNull(cs_net_profit), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_item_sk), IsNotNull(cs_order_number), IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 12] @@ -234,7 +234,7 @@ Input [6]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#4 (33) Filter [codegen id : 12] Input [6]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_net_profit#46] -Condition : ((((((((isnotnull(cs_quantity#44) AND isnotnull(cs_net_paid#45)) AND isnotnull(cs_net_profit#46)) AND (cs_net_profit#46 > 1.00)) AND (cs_net_paid#45 > 0.00)) AND (cs_quantity#44 > 0)) AND isnotnull(cs_item_sk#42)) AND isnotnull(cs_order_number#43)) AND isnotnull(cs_sold_date_sk#41)) +Condition : ((((((((isnotnull(cs_net_profit#46) AND isnotnull(cs_net_paid#45)) AND isnotnull(cs_quantity#44)) AND (cs_net_profit#46 > 1.00)) AND (cs_net_paid#45 > 0.00)) AND (cs_quantity#44 > 0)) AND isnotnull(cs_order_number#43)) AND isnotnull(cs_item_sk#42)) AND isnotnull(cs_sold_date_sk#41)) (34) Project [codegen id : 12] Output [5]: [cs_sold_date_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45] @@ -341,7 +341,7 @@ Input [5]: [item#70, return_ratio#71, currency_ratio#72, return_rank#74, currenc Output [6]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), IsNotNull(ss_net_paid), IsNotNull(ss_net_profit), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 22] @@ -349,7 +349,7 @@ Input [6]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity# (59) Filter [codegen id : 22] Input [6]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82] -Condition : ((((((((isnotnull(ss_quantity#80) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_net_profit#82)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) AND isnotnull(ss_sold_date_sk#77)) +Condition : ((((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) AND isnotnull(ss_sold_date_sk#77)) (60) Project [codegen id : 22] Output [5]: [ss_sold_date_sk#77, ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81] @@ -379,7 +379,7 @@ Arguments: [cast(ss_ticket_number#79 as bigint) ASC NULLS FIRST, cast(ss_item_sk Output [4]: [sr_item_sk#84, sr_ticket_number#85, sr_return_quantity#86, sr_return_amt#87] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] @@ -387,7 +387,7 @@ Input [4]: [sr_item_sk#84, sr_ticket_number#85, sr_return_quantity#86, sr_return (68) Filter [codegen id : 24] Input [4]: [sr_item_sk#84, sr_ticket_number#85, sr_return_quantity#86, sr_return_amt#87] -Condition : (((isnotnull(sr_return_amt#87) AND (sr_return_amt#87 > 10000.00)) AND isnotnull(sr_item_sk#84)) AND isnotnull(sr_ticket_number#85)) +Condition : (((isnotnull(sr_return_amt#87) AND (sr_return_amt#87 > 10000.00)) AND isnotnull(sr_ticket_number#85)) AND isnotnull(sr_item_sk#84)) (69) Exchange Input [4]: [sr_item_sk#84, sr_ticket_number#85, sr_return_quantity#86, sr_return_amt#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt index 7fc20fb4df8d2..ab300ca150457 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (32) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter - Exchange [channel,currency_rank,item,return_rank,return_ratio] #1 + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 WholeStageCodegen (31) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union WholeStageCodegen (10) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (9) @@ -21,47 +21,47 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #2 WholeStageCodegen (7) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,sum,ws_item_sk] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (6) - HashAggregate [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter WholeStageCodegen (3) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #4 + Exchange [ws_order_number,ws_item_sk] #4 WholeStageCodegen (2) - Project [ws_item_sk,ws_net_paid,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_item_sk,ws_net_paid,ws_order_number,ws_quantity,ws_sold_date_sk] - Filter [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter WholeStageCodegen (5) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_item_sk,wr_order_number] #6 + Exchange [wr_order_number,wr_item_sk] #6 WholeStageCodegen (4) - Filter [wr_item_sk,wr_order_number,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] WholeStageCodegen (20) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (19) @@ -73,41 +73,41 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #7 WholeStageCodegen (17) - HashAggregate [cs_item_sk,isEmpty,isEmpty,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #8 WholeStageCodegen (16) - HashAggregate [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter WholeStageCodegen (13) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #9 + Exchange [cs_order_number,cs_item_sk] #9 WholeStageCodegen (12) - Project [cs_item_sk,cs_net_paid,cs_order_number,cs_quantity] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_net_paid,cs_order_number,cs_quantity,cs_sold_date_sk] - Filter [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter WholeStageCodegen (15) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #10 + Exchange [cr_order_number,cr_item_sk] #10 WholeStageCodegen (14) - Filter [cr_item_sk,cr_order_number,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] WholeStageCodegen (30) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (29) @@ -119,35 +119,35 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #11 WholeStageCodegen (27) - HashAggregate [isEmpty,isEmpty,ss_item_sk,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ss_item_sk] #12 WholeStageCodegen (26) - HashAggregate [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter WholeStageCodegen (23) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #13 + Exchange [ss_ticket_number,ss_item_sk] #13 WholeStageCodegen (22) - Project [ss_item_sk,ss_net_paid,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_item_sk,ss_net_paid,ss_quantity,ss_sold_date_sk,ss_ticket_number] - Filter [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter WholeStageCodegen (25) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #14 + Exchange [sr_ticket_number,sr_item_sk] #14 WholeStageCodegen (24) - Filter [sr_item_sk,sr_return_amt,sr_ticket_number] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 0f5821b13b73b..01dc1047953e3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -83,7 +83,7 @@ TakeOrderedAndProject (78) Output [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_sold_date_sk)] +PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 3] @@ -91,7 +91,7 @@ Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, w (3) Filter [codegen id : 3] Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5, ws_net_profit#6] -Condition : ((((((((isnotnull(ws_net_profit#6) AND isnotnull(ws_net_paid#5)) AND isnotnull(ws_quantity#4)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_sold_date_sk#1)) +Condition : ((((((((isnotnull(ws_net_profit#6) AND isnotnull(ws_net_paid#5)) AND isnotnull(ws_quantity#4)) AND (ws_net_profit#6 > 1.00)) AND (ws_net_paid#5 > 0.00)) AND (ws_quantity#4 > 0)) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_item_sk#2)) AND isnotnull(ws_sold_date_sk#1)) (4) Project [codegen id : 3] Output [5]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, ws_net_paid#5] @@ -101,7 +101,7 @@ Input [6]: [ws_sold_date_sk#1, ws_item_sk#2, ws_order_number#3, ws_quantity#4, w Output [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -109,7 +109,7 @@ Input [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt (7) Filter [codegen id : 1] Input [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] -Condition : (((isnotnull(wr_return_amt#10) AND (wr_return_amt#10 > 10000.00)) AND isnotnull(wr_item_sk#7)) AND isnotnull(wr_order_number#8)) +Condition : (((isnotnull(wr_return_amt#10) AND (wr_return_amt#10 > 10000.00)) AND isnotnull(wr_order_number#8)) AND isnotnull(wr_item_sk#7)) (8) BroadcastExchange Input [4]: [wr_item_sk#7, wr_order_number#8, wr_return_quantity#9, wr_return_amt#10] @@ -205,7 +205,7 @@ Input [5]: [item#33, return_ratio#34, currency_ratio#35, return_rank#37, currenc Output [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_net_paid), IsNotNull(cs_net_profit), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_item_sk), IsNotNull(cs_order_number), IsNotNull(cs_sold_date_sk)] +PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 10] @@ -213,7 +213,7 @@ Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#4 (30) Filter [codegen id : 10] Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44, cs_net_profit#45] -Condition : ((((((((isnotnull(cs_quantity#43) AND isnotnull(cs_net_paid#44)) AND isnotnull(cs_net_profit#45)) AND (cs_net_profit#45 > 1.00)) AND (cs_net_paid#44 > 0.00)) AND (cs_quantity#43 > 0)) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_order_number#42)) AND isnotnull(cs_sold_date_sk#40)) +Condition : ((((((((isnotnull(cs_net_profit#45) AND isnotnull(cs_net_paid#44)) AND isnotnull(cs_quantity#43)) AND (cs_net_profit#45 > 1.00)) AND (cs_net_paid#44 > 0.00)) AND (cs_quantity#43 > 0)) AND isnotnull(cs_order_number#42)) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_sold_date_sk#40)) (31) Project [codegen id : 10] Output [5]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#43, cs_net_paid#44] @@ -223,7 +223,7 @@ Input [6]: [cs_sold_date_sk#40, cs_item_sk#41, cs_order_number#42, cs_quantity#4 Output [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 8] @@ -231,7 +231,7 @@ Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_ (34) Filter [codegen id : 8] Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] -Condition : (((isnotnull(cr_return_amount#49) AND (cr_return_amount#49 > 10000.00)) AND isnotnull(cr_item_sk#46)) AND isnotnull(cr_order_number#47)) +Condition : (((isnotnull(cr_return_amount#49) AND (cr_return_amount#49 > 10000.00)) AND isnotnull(cr_order_number#47)) AND isnotnull(cr_item_sk#46)) (35) BroadcastExchange Input [4]: [cr_item_sk#46, cr_order_number#47, cr_return_quantity#48, cr_return_amount#49] @@ -308,7 +308,7 @@ Input [5]: [item#68, return_ratio#69, currency_ratio#70, return_rank#72, currenc Output [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_net_paid), IsNotNull(ss_net_profit), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] +PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 17] @@ -316,7 +316,7 @@ Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity# (53) Filter [codegen id : 17] Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79, ss_net_profit#80] -Condition : ((((((((isnotnull(ss_net_paid#79) AND isnotnull(ss_net_profit#80)) AND isnotnull(ss_quantity#78)) AND (ss_net_profit#80 > 1.00)) AND (ss_net_paid#79 > 0.00)) AND (ss_quantity#78 > 0)) AND isnotnull(ss_ticket_number#77)) AND isnotnull(ss_item_sk#76)) AND isnotnull(ss_sold_date_sk#75)) +Condition : ((((((((isnotnull(ss_net_profit#80) AND isnotnull(ss_net_paid#79)) AND isnotnull(ss_quantity#78)) AND (ss_net_profit#80 > 1.00)) AND (ss_net_paid#79 > 0.00)) AND (ss_quantity#78 > 0)) AND isnotnull(ss_ticket_number#77)) AND isnotnull(ss_item_sk#76)) AND isnotnull(ss_sold_date_sk#75)) (54) Project [codegen id : 17] Output [5]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity#78, ss_net_paid#79] @@ -326,7 +326,7 @@ Input [6]: [ss_sold_date_sk#75, ss_item_sk#76, ss_ticket_number#77, ss_quantity# Output [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 15] @@ -334,7 +334,7 @@ Input [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return (57) Filter [codegen id : 15] Input [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] -Condition : (((isnotnull(sr_return_amt#84) AND (sr_return_amt#84 > 10000.00)) AND isnotnull(sr_item_sk#81)) AND isnotnull(sr_ticket_number#82)) +Condition : (((isnotnull(sr_return_amt#84) AND (sr_return_amt#84 > 10000.00)) AND isnotnull(sr_ticket_number#82)) AND isnotnull(sr_item_sk#81)) (58) BroadcastExchange Input [4]: [sr_item_sk#81, sr_ticket_number#82, sr_return_quantity#83, sr_return_amt#84] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index acba83ae8e411..c15f2394e1a44 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] WholeStageCodegen (23) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter - Exchange [channel,currency_rank,item,return_rank,return_ratio] #1 + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 WholeStageCodegen (22) - HashAggregate [channel,currency_rank,item,return_rank,return_ratio] + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union WholeStageCodegen (7) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (6) @@ -21,38 +21,38 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #2 WholeStageCodegen (4) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,sum,ws_item_sk] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(wr_return_quantity, 0) as bigint)),sum(cast(coalesce(ws_quantity, 0) as bigint)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 WholeStageCodegen (3) - HashAggregate [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_return_amt,wr_return_quantity,ws_item_sk,ws_net_paid,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Project [ws_item_sk,ws_net_paid,ws_order_number,ws_quantity,ws_sold_date_sk] - Filter [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_net_profit,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [wr_item_sk,wr_order_number,wr_return_amt] + Filter [wr_return_amt,wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Project [d_date_sk] - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (14) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (13) @@ -64,32 +64,32 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #6 WholeStageCodegen (11) - HashAggregate [cs_item_sk,isEmpty,isEmpty,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(cr_return_quantity, 0) as bigint)),sum(cast(coalesce(cs_quantity, 0) as bigint)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #7 WholeStageCodegen (10) - HashAggregate [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity] + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_return_amount,cr_return_quantity,cs_item_sk,cs_net_paid,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_item_sk,cs_net_paid,cs_order_number,cs_quantity,cs_sold_date_sk] - Filter [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_net_paid,cs_net_profit,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit] InputAdapter BroadcastExchange #8 WholeStageCodegen (8) - Filter [cr_item_sk,cr_order_number,cr_return_amount] + Filter [cr_return_amount,cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (21) - Project [currency_rank,item,return_rank,return_ratio] - Filter [currency_rank,return_rank] + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] WholeStageCodegen (20) @@ -101,26 +101,26 @@ TakeOrderedAndProject [channel,currency_rank,item,return_rank,return_ratio] InputAdapter Exchange #9 WholeStageCodegen (18) - HashAggregate [isEmpty,isEmpty,ss_item_sk,sum,sum,sum,sum] [currency_ratio,isEmpty,isEmpty,item,return_ratio,sum,sum,sum,sum,sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(cast(coalesce(sr_return_quantity, 0) as bigint)),sum(cast(coalesce(ss_quantity, 0) as bigint)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ss_item_sk] #10 WholeStageCodegen (17) - HashAggregate [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [sr_return_amt,sr_return_quantity,ss_item_sk,ss_net_paid,ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Project [ss_item_sk,ss_net_paid,ss_quantity,ss_sold_date_sk,ss_ticket_number] - Filter [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_net_paid,ss_net_profit,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit] InputAdapter BroadcastExchange #11 WholeStageCodegen (15) - Filter [sr_item_sk,sr_return_amt,sr_ticket_number] + Filter [sr_return_amt,sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] InputAdapter ReusedExchange [d_date_sk] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt index 245ad9d53f2cf..27e0d254e0d7f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/simplified.txt @@ -1,32 +1,32 @@ -TakeOrderedAndProject [d_date,item_sk,store_cumulative,store_sales,web_cumulative,web_sales] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] WholeStageCodegen (63) - Filter [store_cumulative,web_cumulative] - HashAggregate [d_date,item_sk,max,max,store_sales,web_sales] [max,max,max(store_sales),max(web_sales),store_cumulative,web_cumulative] - HashAggregate [d_date,item_sk,store_sales,store_sales,web_sales,web_sales] [max,max,max,max] - Project [d_date,item_sk,store_sales,store_sales,web_sales,web_sales] + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] SortMergeJoin [item_sk,item_sk,rk,rk] InputAdapter WholeStageCodegen (31) Filter [rk] InputAdapter - Window [d_date,item_sk] + Window [item_sk,d_date] WholeStageCodegen (30) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 WholeStageCodegen (29) - Project [cume_sales,cume_sales,d_date,d_date,item_sk,item_sk] + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] Filter [item_sk,item_sk] InputAdapter - SortMergeJoin [d_date,d_date,item_sk,item_sk] + SortMergeJoin [item_sk,d_date,item_sk,d_date] WholeStageCodegen (14) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #2 + Exchange [item_sk,d_date] #2 WholeStageCodegen (13) - HashAggregate [d_date,isEmpty,item_sk,sum,sumws] [cume_sales,isEmpty,sum,sum(sumws)] - HashAggregate [d_date,item_sk,sumws,sumws] [isEmpty,isEmpty,sum,sum] - Project [d_date,item_sk,sumws,sumws] + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] SortMergeJoin [item_sk,item_sk,rk,rk] InputAdapter WholeStageCodegen (6) @@ -34,56 +34,56 @@ TakeOrderedAndProject [d_date,item_sk,store_cumulative,store_sales,web_cumulativ InputAdapter Exchange [item_sk] #3 WholeStageCodegen (5) - Project [d_date,item_sk,rk,sumws] + Project [item_sk,d_date,sumws,rk] Filter [rk] InputAdapter - Window [d_date,ws_item_sk] + Window [ws_item_sk,d_date] WholeStageCodegen (4) - Sort [d_date,ws_item_sk] + Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) - HashAggregate [d_date,sum,ws_item_sk] [item_sk,sum,sum(UnscaledValue(ws_sales_price)),sumws] + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] InputAdapter - Exchange [d_date,ws_item_sk] #5 + Exchange [ws_item_sk,d_date] #5 WholeStageCodegen (2) - HashAggregate [d_date,ws_item_sk,ws_sales_price] [sum,sum] - Project [d_date,ws_item_sk,ws_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter WholeStageCodegen (12) Sort [item_sk] InputAdapter Exchange [item_sk] #7 WholeStageCodegen (11) - Project [item_sk,rk,sumws] + Project [item_sk,sumws,rk] Filter [rk] InputAdapter - Window [d_date,ws_item_sk] + Window [ws_item_sk,d_date] WholeStageCodegen (10) - Sort [d_date,ws_item_sk] + Sort [ws_item_sk,d_date] InputAdapter - ReusedExchange [d_date,item_sk,sumws,ws_item_sk] #4 + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 WholeStageCodegen (28) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #8 + Exchange [item_sk,d_date] #8 WholeStageCodegen (27) - HashAggregate [d_date,isEmpty,item_sk,sum,sumss] [cume_sales,isEmpty,sum,sum(sumss)] - HashAggregate [d_date,item_sk,sumss,sumss] [isEmpty,isEmpty,sum,sum] - Project [d_date,item_sk,sumss,sumss] + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] SortMergeJoin [item_sk,item_sk,rk,rk] InputAdapter WholeStageCodegen (20) @@ -91,49 +91,49 @@ TakeOrderedAndProject [d_date,item_sk,store_cumulative,store_sales,web_cumulativ InputAdapter Exchange [item_sk] #9 WholeStageCodegen (19) - Project [d_date,item_sk,rk,sumss] + Project [item_sk,d_date,sumss,rk] Filter [rk] InputAdapter - Window [d_date,ss_item_sk] + Window [ss_item_sk,d_date] WholeStageCodegen (18) - Sort [d_date,ss_item_sk] + Sort [ss_item_sk,d_date] InputAdapter Exchange [ss_item_sk] #10 WholeStageCodegen (17) - HashAggregate [d_date,ss_item_sk,sum] [item_sk,sum,sum(UnscaledValue(ss_sales_price)),sumss] + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] InputAdapter - Exchange [d_date,ss_item_sk] #11 + Exchange [ss_item_sk,d_date] #11 WholeStageCodegen (16) - HashAggregate [d_date,ss_item_sk,ss_sales_price] [sum,sum] - Project [d_date,ss_item_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] InputAdapter - ReusedExchange [d_date,d_date_sk] #6 + ReusedExchange [d_date_sk,d_date] #6 InputAdapter WholeStageCodegen (26) Sort [item_sk] InputAdapter Exchange [item_sk] #12 WholeStageCodegen (25) - Project [item_sk,rk,sumss] + Project [item_sk,sumss,rk] Filter [rk] InputAdapter - Window [d_date,ss_item_sk] + Window [ss_item_sk,d_date] WholeStageCodegen (24) - Sort [d_date,ss_item_sk] + Sort [ss_item_sk,d_date] InputAdapter - ReusedExchange [d_date,item_sk,ss_item_sk,sumss] #10 + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter WholeStageCodegen (62) - Project [item_sk,rk,store_sales,web_sales] + Project [item_sk,web_sales,store_sales,rk] Filter [rk] InputAdapter - Window [d_date,item_sk] + Window [item_sk,d_date] WholeStageCodegen (61) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - ReusedExchange [d_date,item_sk,store_sales,web_sales] #1 + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index f7f09fd64ee68..c17a3085b030c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,126 +1,126 @@ -TakeOrderedAndProject [d_date,item_sk,store_cumulative,store_sales,web_cumulative,web_sales] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] WholeStageCodegen (54) - Filter [store_cumulative,web_cumulative] - HashAggregate [d_date,item_sk,max,max,store_sales,web_sales] [max,max,max(store_sales),max(web_sales),store_cumulative,web_cumulative] - HashAggregate [d_date,item_sk,store_sales,store_sales,web_sales,web_sales] [max,max,max,max] - Project [d_date,item_sk,store_sales,store_sales,web_sales,web_sales] + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] BroadcastHashJoin [item_sk,item_sk,rk,rk] Filter [rk] InputAdapter - Window [d_date,item_sk] + Window [item_sk,d_date] WholeStageCodegen (26) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 WholeStageCodegen (25) - Project [cume_sales,cume_sales,d_date,d_date,item_sk,item_sk] + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] Filter [item_sk,item_sk] InputAdapter - SortMergeJoin [d_date,d_date,item_sk,item_sk] + SortMergeJoin [item_sk,d_date,item_sk,d_date] WholeStageCodegen (12) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #2 + Exchange [item_sk,d_date] #2 WholeStageCodegen (11) - HashAggregate [d_date,isEmpty,item_sk,sum,sumws] [cume_sales,isEmpty,sum,sum(sumws)] + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] InputAdapter - Exchange [d_date,item_sk,sumws] #3 + Exchange [item_sk,d_date,sumws] #3 WholeStageCodegen (10) - HashAggregate [d_date,item_sk,sumws,sumws] [isEmpty,isEmpty,sum,sum] - Project [d_date,item_sk,sumws,sumws] + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [d_date,item_sk,rk,sumws] + Project [item_sk,d_date,sumws,rk] Filter [rk] InputAdapter - Window [d_date,ws_item_sk] + Window [ws_item_sk,d_date] WholeStageCodegen (4) - Sort [d_date,ws_item_sk] + Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) - HashAggregate [d_date,sum,ws_item_sk] [item_sk,sum,sum(UnscaledValue(ws_sales_price)),sumws] + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] InputAdapter - Exchange [d_date,ws_item_sk] #5 + Exchange [ws_item_sk,d_date] #5 WholeStageCodegen (2) - HashAggregate [d_date,ws_item_sk,ws_sales_price] [sum,sum] - Project [d_date,ws_item_sk,ws_sales_price] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Project [d_date,d_date_sk] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_date] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_month_seq] + Scan parquet default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter BroadcastExchange #7 WholeStageCodegen (9) - Project [item_sk,rk,sumws] + Project [item_sk,sumws,rk] Filter [rk] InputAdapter - Window [d_date,ws_item_sk] + Window [ws_item_sk,d_date] WholeStageCodegen (8) - Sort [d_date,ws_item_sk] + Sort [ws_item_sk,d_date] InputAdapter - ReusedExchange [d_date,item_sk,sumws,ws_item_sk] #4 + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 WholeStageCodegen (24) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - Exchange [d_date,item_sk] #8 + Exchange [item_sk,d_date] #8 WholeStageCodegen (23) - HashAggregate [d_date,isEmpty,item_sk,sum,sumss] [cume_sales,isEmpty,sum,sum(sumss)] + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] InputAdapter - Exchange [d_date,item_sk,sumss] #9 + Exchange [item_sk,d_date,sumss] #9 WholeStageCodegen (22) - HashAggregate [d_date,item_sk,sumss,sumss] [isEmpty,isEmpty,sum,sum] - Project [d_date,item_sk,sumss,sumss] + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [d_date,item_sk,rk,sumss] + Project [item_sk,d_date,sumss,rk] Filter [rk] InputAdapter - Window [d_date,ss_item_sk] + Window [ss_item_sk,d_date] WholeStageCodegen (16) - Sort [d_date,ss_item_sk] + Sort [ss_item_sk,d_date] InputAdapter Exchange [ss_item_sk] #10 WholeStageCodegen (15) - HashAggregate [d_date,ss_item_sk,sum] [item_sk,sum,sum(UnscaledValue(ss_sales_price)),sumss] + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] InputAdapter - Exchange [d_date,ss_item_sk] #11 + Exchange [ss_item_sk,d_date] #11 WholeStageCodegen (14) - HashAggregate [d_date,ss_item_sk,ss_sales_price] [sum,sum] - Project [d_date,ss_item_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_sales_price] InputAdapter - ReusedExchange [d_date,d_date_sk] #6 + ReusedExchange [d_date_sk,d_date] #6 InputAdapter BroadcastExchange #12 WholeStageCodegen (21) - Project [item_sk,rk,sumss] + Project [item_sk,sumss,rk] Filter [rk] InputAdapter - Window [d_date,ss_item_sk] + Window [ss_item_sk,d_date] WholeStageCodegen (20) - Sort [d_date,ss_item_sk] + Sort [ss_item_sk,d_date] InputAdapter - ReusedExchange [d_date,item_sk,ss_item_sk,sumss] #10 + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 WholeStageCodegen (53) - Project [item_sk,rk,store_sales,web_sales] + Project [item_sk,web_sales,store_sales,rk] Filter [rk] InputAdapter - Window [d_date,item_sk] + Window [item_sk,d_date] WholeStageCodegen (52) - Sort [d_date,item_sk] + Sort [item_sk,d_date] InputAdapter - ReusedExchange [d_date,item_sk,store_sales,web_sales] #1 + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index 105de3e077e6a..e9f2b20fe64b2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -139,7 +139,7 @@ Arguments: [cs_item_sk#3 ASC NULLS FIRST], false, 0 Output [3]: [i_item_sk#13, i_brand#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand), IsNotNull(i_category)] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] ReadSchema: struct (19) ColumnarToRow [codegen id : 5] @@ -147,7 +147,7 @@ Input [3]: [i_item_sk#13, i_brand#14, i_category#15] (20) Filter [codegen id : 5] Input [3]: [i_item_sk#13, i_brand#14, i_category#15] -Condition : ((isnotnull(i_item_sk#13) AND isnotnull(i_brand#14)) AND isnotnull(i_category#15)) +Condition : ((isnotnull(i_item_sk#13) AND isnotnull(i_category#15)) AND isnotnull(i_brand#14)) (21) Exchange Input [3]: [i_item_sk#13, i_brand#14, i_category#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt index eb386bc583093..44b0529577849 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/simplified.txt @@ -1,39 +1,39 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,psum,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (37) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,sum_sales,sum_sales,sum_sales] - SortMergeJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (25) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,rn,sum_sales,sum_sales] - SortMergeJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + SortMergeJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] InputAdapter WholeStageCodegen (13) - Sort [cc_name,i_brand,i_category,rn] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [cc_name,i_brand,i_category,rn] #1 + Exchange [i_category,i_brand,cc_name,rn] #1 WholeStageCodegen (12) - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (11) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #2 + Exchange [i_category,i_brand,cc_name] #2 WholeStageCodegen (10) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,sum_sales] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,cc_name,d_year,i_brand,i_category] + Window [_w0,i_category,i_brand,cc_name,d_year] WholeStageCodegen (9) - Sort [cc_name,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year] InputAdapter - Exchange [cc_name,d_year,i_brand,i_category] #3 + Exchange [i_category,i_brand,cc_name,d_year] #3 WholeStageCodegen (8) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [_w0,sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [cc_name,d_moy,d_year,i_brand,i_category] #4 + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #4 WholeStageCodegen (7) - HashAggregate [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] [sum,sum] - Project [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) @@ -41,21 +41,21 @@ TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,ps InputAdapter Exchange [cs_item_sk] #5 WholeStageCodegen (3) - Project [cc_name,cs_item_sk,cs_sales_price,d_moy,d_year] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_moy,d_year] + Project [cs_item_sk,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_item_sk,cs_sales_price,d_year,d_moy] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_call_center_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_item_sk,cs_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) @@ -69,39 +69,39 @@ TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,ps InputAdapter Exchange [i_item_sk] #8 WholeStageCodegen (5) - Filter [i_brand,i_category,i_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter WholeStageCodegen (24) - Sort [cc_name,i_brand,i_category,rn] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [cc_name,i_brand,i_category,rn] #9 + Exchange [i_category,i_brand,cc_name,rn] #9 WholeStageCodegen (23) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (22) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #10 + Exchange [i_category,i_brand,cc_name] #10 WholeStageCodegen (21) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum] #4 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #4 InputAdapter WholeStageCodegen (36) - Sort [cc_name,i_brand,i_category,rn] + Sort [i_category,i_brand,cc_name,rn] InputAdapter - Exchange [cc_name,i_brand,i_category,rn] #11 + Exchange [i_category,i_brand,cc_name,rn] #11 WholeStageCodegen (35) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (34) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum_sales] #10 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index 524ddbc102c11..fcd44a9393007 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -195,7 +195,7 @@ Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#14, d_year#10, d_moy#11, sum_sales#20, avg_monthly_sales#23, rn#25] -Condition : (((((isnotnull(avg_monthly_sales#23) AND isnotnull(d_year#10)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#23 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#25)) +Condition : (((((isnotnull(d_year#10) AND isnotnull(avg_monthly_sales#23)) AND (d_year#10 = 1999)) AND (avg_monthly_sales#23 > 0.000000)) AND (CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16), true) ELSE null END > 0.1000000000000000)) AND isnotnull(rn#25)) (33) ReusedExchange [Reuses operator id: 23] Output [6]: [i_category#26, i_brand#27, cc_name#28, d_year#29, d_moy#30, sum#31] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 6a0135b852696..aafe3462682cf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,54 +1,54 @@ -TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,psum,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] WholeStageCodegen (23) - Project [avg_monthly_sales,d_moy,d_year,i_brand,i_category,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,rn,sum_sales,sum_sales] - BroadcastHashJoin [cc_name,cc_name,i_brand,i_brand,i_category,i_category,rn,rn] - Filter [avg_monthly_sales,d_year,rn,sum_sales] + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Filter [d_year,avg_monthly_sales,sum_sales,rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (8) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #1 + Exchange [i_category,i_brand,cc_name] #1 WholeStageCodegen (7) - Project [avg_monthly_sales,cc_name,d_moy,d_year,i_brand,i_category,sum_sales] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales] InputAdapter - Window [_w0,cc_name,d_year,i_brand,i_category] + Window [_w0,i_category,i_brand,cc_name,d_year] WholeStageCodegen (6) - Sort [cc_name,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year] InputAdapter - Exchange [cc_name,d_year,i_brand,i_category] #2 + Exchange [i_category,i_brand,cc_name,d_year] #2 WholeStageCodegen (5) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [_w0,sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter - Exchange [cc_name,d_moy,d_year,i_brand,i_category] #3 + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #3 WholeStageCodegen (4) - HashAggregate [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] [sum,sum] - Project [cc_name,cs_sales_price,d_moy,d_year,i_brand,i_category] - BroadcastHashJoin [cc_call_center_sk,cs_call_center_sk] - Project [cs_call_center_sk,cs_sales_price,d_moy,d_year,i_brand,i_category] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_call_center_sk,cs_sales_price,cs_sold_date_sk,i_brand,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [i_brand,i_category,i_item_sk] + Project [i_brand,i_category,cs_sold_date_sk,cs_call_center_sk,cs_sales_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + Filter [i_item_sk,i_category,i_brand] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_item_sk] + Scan parquet default.item [i_item_sk,i_brand,i_category] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [cs_call_center_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_call_center_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_item_sk,cs_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Filter [d_date_sk,d_moy,d_year] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_moy,d_year] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) @@ -59,26 +59,26 @@ TakeOrderedAndProject [avg_monthly_sales,d_moy,d_year,i_brand,i_category,nsum,ps InputAdapter BroadcastExchange #7 WholeStageCodegen (15) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (14) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [cc_name,i_brand,i_category] #8 + Exchange [i_category,i_brand,cc_name] #8 WholeStageCodegen (13) - HashAggregate [cc_name,d_moy,d_year,i_brand,i_category,sum] [sum,sum(UnscaledValue(cs_sales_price)),sum_sales] + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum] #3 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #3 InputAdapter BroadcastExchange #9 WholeStageCodegen (22) - Project [cc_name,i_brand,i_category,rn,sum_sales] + Project [i_category,i_brand,cc_name,sum_sales,rn] Filter [rn] InputAdapter - Window [cc_name,d_moy,d_year,i_brand,i_category] + Window [d_year,d_moy,i_category,i_brand,cc_name] WholeStageCodegen (21) - Sort [cc_name,d_moy,d_year,i_brand,i_category] + Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [cc_name,d_moy,d_year,i_brand,i_category,sum_sales] #8 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index 209f65e8d333e..81b4178b7a9ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -1,52 +1,52 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (78) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #1 + Exchange [channel,id,sales,returns,profit] #1 WholeStageCodegen (77) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (51) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #2 + Exchange [channel,id,sales,returns,profit] #2 WholeStageCodegen (50) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (24) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #3 WholeStageCodegen (23) - HashAggregate [channel,id,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [channel,id,profit,returns,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + HashAggregate [s_store_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] InputAdapter Exchange [s_store_id] #4 WholeStageCodegen (5) - HashAggregate [net_loss,profit,return_amt,s_store_id,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,s_store_id,sales_price] - BroadcastHashJoin [s_store_sk,store_sk] - Project [net_loss,profit,return_amt,sales_price,store_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (1) - Project [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] WholeStageCodegen (2) - Project [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -54,38 +54,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [channel,id,profit,returns,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [cp_catalog_page_id] #7 WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] - BroadcastHashJoin [cp_catalog_page_sk,page_sk] - Project [net_loss,page_sk,profit,return_amt,sales_price] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - Filter [cs_catalog_page_sk,cs_sold_date_sk] + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_sold_date_sk,cs_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] - Filter [cr_catalog_page_sk,cr_returned_date_sk] + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_returned_date_sk,cr_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -94,27 +94,27 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (22) - HashAggregate [sum,sum,sum,sum,web_site_id] [channel,id,profit,returns,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [web_site_id] #9 WholeStageCodegen (21) - HashAggregate [net_loss,profit,return_amt,sales_price,web_site_id] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,sales_price,web_site_id] - BroadcastHashJoin [web_site_sk,wsr_web_site_sk] - Project [net_loss,profit,return_amt,sales_price,wsr_web_site_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (13) - Project [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_sold_date_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit] WholeStageCodegen (18) - Project [wr_net_loss,wr_return_amt,wr_returned_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter WholeStageCodegen (15) @@ -125,7 +125,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [wr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter WholeStageCodegen (17) Sort [ws_item_sk,ws_order_number] @@ -135,7 +135,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_order_number,ws_web_site_sk] + Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -144,22 +144,22 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] WholeStageCodegen (49) - HashAggregate [channel,isEmpty,isEmpty,isEmpty,sum,sum,sum] [id,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum(profit),sum(profit),sum(returns),sum(returns),sum(sales),sum(sales)] + 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] InputAdapter Exchange [channel] #13 WholeStageCodegen (48) - HashAggregate [channel,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 WholeStageCodegen (76) - HashAggregate [isEmpty,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum(profit),sum(profit),sum(returns),sum(returns),sum(sales),sum(sales)] + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #14 WholeStageCodegen (75) - HashAggregate [profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index 77d7d6f938665..6bb223e2f4488 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -1,52 +1,52 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (69) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #1 + Exchange [channel,id,sales,returns,profit] #1 WholeStageCodegen (68) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (45) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #2 + Exchange [channel,id,sales,returns,profit] #2 WholeStageCodegen (44) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (21) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #3 WholeStageCodegen (20) - HashAggregate [channel,id,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [channel,id,profit,returns,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + HashAggregate [s_store_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] InputAdapter Exchange [s_store_id] #4 WholeStageCodegen (5) - HashAggregate [net_loss,profit,return_amt,s_store_id,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,s_store_id,sales_price] - BroadcastHashJoin [s_store_sk,store_sk] - Project [net_loss,profit,return_amt,sales_price,store_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (1) - Project [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] WholeStageCodegen (2) - Project [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) @@ -54,38 +54,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [channel,id,profit,returns,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [cp_catalog_page_id] #7 WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,net_loss,profit,return_amt,sales_price] - BroadcastHashJoin [cp_catalog_page_sk,page_sk] - Project [net_loss,page_sk,profit,return_amt,sales_price] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - Filter [cs_catalog_page_sk,cs_sold_date_sk] + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_sold_date_sk,cs_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] - Filter [cr_catalog_page_sk,cr_returned_date_sk] + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_returned_date_sk,cr_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_catalog_page_sk,cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -94,39 +94,39 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (19) - HashAggregate [sum,sum,sum,sum,web_site_id] [channel,id,profit,returns,sales,sum,sum,sum,sum,sum(UnscaledValue(net_loss)),sum(UnscaledValue(profit)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(sales_price))] + 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] InputAdapter Exchange [web_site_id] #9 WholeStageCodegen (18) - HashAggregate [net_loss,profit,return_amt,sales_price,web_site_id] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [net_loss,profit,return_amt,sales_price,web_site_id] - BroadcastHashJoin [web_site_sk,wsr_web_site_sk] - Project [net_loss,profit,return_amt,sales_price,wsr_web_site_sk] - BroadcastHashJoin [d_date_sk,date_sk] + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union WholeStageCodegen (13) - Project [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_sold_date_sk,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_site_sk,ws_ext_sales_price,ws_net_profit] WholeStageCodegen (15) - Project [wr_net_loss,wr_return_amt,wr_returned_date_sk,ws_web_site_sk] + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] Filter [wr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt,wr_returned_date_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter BroadcastExchange #10 WholeStageCodegen (14) Filter [ws_item_sk,ws_order_number,ws_web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_order_number,ws_web_site_sk] + Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -135,22 +135,22 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] WholeStageCodegen (43) - HashAggregate [channel,isEmpty,isEmpty,isEmpty,sum,sum,sum] [id,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum(profit),sum(profit),sum(returns),sum(returns),sum(sales),sum(sales)] + 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] InputAdapter Exchange [channel] #12 WholeStageCodegen (42) - HashAggregate [channel,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 WholeStageCodegen (67) - HashAggregate [isEmpty,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum(profit),sum(profit),sum(returns),sum(returns),sum(sales),sum(sales)] + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #13 WholeStageCodegen (66) - HashAggregate [profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index 924f669212155..2700741b82c04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [ca_state,cnt,state] +TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (14) - Project [ca_state,cnt,state] + Project [state,cnt,ca_state] Filter [count(1)] - HashAggregate [ca_state,count] [cnt,count,count(1),count(1),state] + HashAggregate [ca_state,count] [count(1),state,cnt,count(1),count] InputAdapter Exchange [ca_state] #1 WholeStageCodegen (13) HashAggregate [ca_state] [count,count] Project [ca_state] - SortMergeJoin [c_customer_sk,ss_customer_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (6) Sort [ss_customer_sk] @@ -16,18 +16,18 @@ TakeOrderedAndProject [ca_state,cnt,state] Exchange [ss_customer_sk] #2 WholeStageCodegen (5) Project [ss_customer_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ss_customer_sk,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] Subquery #1 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -36,10 +36,10 @@ TakeOrderedAndProject [ca_state,cnt,state] WholeStageCodegen (1) HashAggregate [d_month_seq] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -47,32 +47,32 @@ TakeOrderedAndProject [ca_state,cnt,state] BroadcastExchange #5 WholeStageCodegen (4) Project [i_item_sk] - Filter [avg(i_current_price),i_current_price] + Filter [i_current_price,avg(i_current_price)] BroadcastHashJoin [i_category,i_category] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - HashAggregate [count,i_category,sum] [avg(UnscaledValue(i_current_price)),avg(i_current_price),count,i_category,sum] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #7 WholeStageCodegen (2) - HashAggregate [i_category,i_current_price] [count,count,sum,sum] + HashAggregate [i_category,i_current_price] [sum,count,sum,count] Filter [i_category] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price] + Scan parquet default.item [i_current_price,i_category] InputAdapter WholeStageCodegen (12) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #8 WholeStageCodegen (11) - Project [c_customer_sk,ca_state] - SortMergeJoin [c_current_addr_sk,ca_address_sk] + Project [ca_state,c_customer_sk] + SortMergeJoin [ca_address_sk,c_current_addr_sk] InputAdapter WholeStageCodegen (8) Sort [ca_address_sk] @@ -92,4 +92,4 @@ TakeOrderedAndProject [ca_state,cnt,state] Filter [c_current_addr_sk,c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index affe3f93d6e73..8502635ba4d35 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,20 +1,20 @@ -TakeOrderedAndProject [ca_state,cnt,state] +TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (8) - Project [ca_state,cnt,state] + Project [state,cnt,ca_state] Filter [count(1)] - HashAggregate [ca_state,count] [cnt,count,count(1),count(1),state] + HashAggregate [ca_state,count] [count(1),state,cnt,count(1),count] InputAdapter Exchange [ca_state] #1 WholeStageCodegen (7) HashAggregate [ca_state] [count,count] Project [ca_state] - BroadcastHashJoin [i_item_sk,ss_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] Project [ca_state,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_sold_date_sk,ss_item_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [c_customer_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] Filter [ca_address_sk] ColumnarToRow InputAdapter @@ -25,19 +25,19 @@ TakeOrderedAndProject [ca_state,cnt,state] Filter [c_current_addr_sk,c_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_customer_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] Subquery #1 WholeStageCodegen (2) HashAggregate [d_month_seq] @@ -46,10 +46,10 @@ TakeOrderedAndProject [ca_state,cnt,state] WholeStageCodegen (1) HashAggregate [d_month_seq] Project [d_month_seq] - Filter [d_moy,d_year] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_month_seq,d_moy,d_year] + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -57,21 +57,21 @@ TakeOrderedAndProject [ca_state,cnt,state] BroadcastExchange #6 WholeStageCodegen (6) Project [i_item_sk] - Filter [avg(i_current_price),i_current_price] + Filter [i_current_price,avg(i_current_price)] BroadcastHashJoin [i_category,i_category] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_category] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [count,i_category,sum] [avg(UnscaledValue(i_current_price)),avg(i_current_price),count,i_category,sum] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) - HashAggregate [i_category,i_current_price] [count,count,sum,sum] + HashAggregate [i_category,i_current_price] [sum,count,sum,count] Filter [i_category] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_current_price] + Scan parquet default.item [i_current_price,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt index 744936e3d80d0..ae75932bcb5e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt @@ -386,7 +386,7 @@ Input [13]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, s Output [3]: [s_store_sk#42, s_store_name#43, s_zip#44] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip), IsNotNull(s_store_name)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct (40) ColumnarToRow [codegen id : 12] @@ -394,7 +394,7 @@ Input [3]: [s_store_sk#42, s_store_name#43, s_zip#44] (41) Filter [codegen id : 12] Input [3]: [s_store_sk#42, s_store_name#43, s_zip#44] -Condition : ((isnotnull(s_store_sk#42) AND isnotnull(s_zip#44)) AND isnotnull(s_store_name#43)) +Condition : ((isnotnull(s_store_sk#42) AND isnotnull(s_store_name#43)) AND isnotnull(s_zip#44)) (42) BroadcastExchange Input [3]: [s_store_sk#42, s_store_name#43, s_zip#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt index b44625037841a..84705518d092b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt @@ -1,28 +1,28 @@ WholeStageCodegen (88) - Sort [cnt,product_name,s1,s1,store_name] + Sort [product_name,store_name,cnt,s1,s1] InputAdapter - Exchange [cnt,product_name,s1,s1,store_name] #1 + Exchange [product_name,store_name,cnt,s1,s1] #1 WholeStageCodegen (87) - Project [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,cnt,product_name,s1,s1,s2,s2,s3,s3,store_name,store_zip,syear,syear] - SortMergeJoin [cnt,cnt,item_sk,item_sk,store_name,store_name,store_zip,store_zip] + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] InputAdapter WholeStageCodegen (43) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #2 WholeStageCodegen (42) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,count,count(1),item_sk,product_name,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #3 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #3 WholeStageCodegen (41) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (35) @@ -30,21 +30,21 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_addr_sk] #4 WholeStageCodegen (34) - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (31) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #5 WholeStageCodegen (30) - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] InputAdapter WholeStageCodegen (24) @@ -52,44 +52,44 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_cdemo_sk] #6 WholeStageCodegen (23) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [cd_demo_sk,ss_cdemo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + SortMergeJoin [ss_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (20) Sort [ss_cdemo_sk] InputAdapter Exchange [ss_cdemo_sk] #7 WholeStageCodegen (19) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (14) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (13) - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (2) Sort [ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_item_sk,ss_ticket_number] #9 WholeStageCodegen (1) - Filter [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] @@ -104,14 +104,14 @@ WholeStageCodegen (88) BroadcastExchange #11 WholeStageCodegen (10) Project [cs_item_sk] - Filter [sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price)] - HashAggregate [cs_item_sk,isEmpty,sum,sum] [isEmpty,sum,sum,sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(UnscaledValue(cs_ext_list_price)),sum(cs_ext_list_price)] + Filter [sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true))] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #12 WholeStageCodegen (9) - HashAggregate [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] [isEmpty,isEmpty,sum,sum,sum,sum] - Project [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter WholeStageCodegen (6) Sort [cs_item_sk,cs_order_number] @@ -121,7 +121,7 @@ WholeStageCodegen (88) Filter [cs_item_sk,cs_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_list_price,cs_item_sk,cs_order_number] + Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price] InputAdapter WholeStageCodegen (8) Sort [cr_item_sk,cr_order_number] @@ -135,27 +135,27 @@ WholeStageCodegen (88) InputAdapter BroadcastExchange #15 WholeStageCodegen (11) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #16 WholeStageCodegen (12) - Filter [s_store_name,s_store_sk,s_zip] + Filter [s_store_sk,s_store_name,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter WholeStageCodegen (16) Sort [c_customer_sk] InputAdapter Exchange [c_customer_sk] #17 WholeStageCodegen (15) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #18 WholeStageCodegen (17) @@ -205,12 +205,12 @@ WholeStageCodegen (88) Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter WholeStageCodegen (37) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #22 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #22 InputAdapter BroadcastExchange #23 WholeStageCodegen (38) @@ -224,28 +224,28 @@ WholeStageCodegen (88) BroadcastExchange #24 WholeStageCodegen (40) Project [i_item_sk,i_product_name] - Filter [i_color,i_current_price,i_item_sk] + Filter [i_current_price,i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter WholeStageCodegen (86) Sort [item_sk,store_name,store_zip] InputAdapter Exchange [item_sk,store_name,store_zip] #25 WholeStageCodegen (85) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [cnt,count,count(1),item_sk,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #26 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #26 WholeStageCodegen (84) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] SortMergeJoin [c_current_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (78) @@ -253,21 +253,21 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_addr_sk] #27 WholeStageCodegen (77) - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - SortMergeJoin [ca_address_sk,ss_addr_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + SortMergeJoin [ss_addr_sk,ca_address_sk] InputAdapter WholeStageCodegen (74) Sort [ss_addr_sk] InputAdapter Exchange [ss_addr_sk] #28 WholeStageCodegen (73) - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] SortMergeJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] InputAdapter WholeStageCodegen (67) @@ -275,39 +275,39 @@ WholeStageCodegen (88) InputAdapter Exchange [c_current_cdemo_sk] #29 WholeStageCodegen (66) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [cd_demo_sk,ss_cdemo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + SortMergeJoin [ss_cdemo_sk,cd_demo_sk] InputAdapter WholeStageCodegen (63) Sort [ss_cdemo_sk] InputAdapter Exchange [ss_cdemo_sk] #30 WholeStageCodegen (62) - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (57) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #31 WholeStageCodegen (56) - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] InputAdapter WholeStageCodegen (45) Sort [ss_item_sk,ss_ticket_number] InputAdapter - ReusedExchange [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] #9 + ReusedExchange [ss_sold_date_sk,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] #9 InputAdapter WholeStageCodegen (47) Sort [sr_item_sk,sr_ticket_number] @@ -318,17 +318,17 @@ WholeStageCodegen (88) InputAdapter BroadcastExchange #32 WholeStageCodegen (54) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [s_store_name,s_store_sk,s_zip] #16 + ReusedExchange [s_store_sk,s_store_name,s_zip] #16 InputAdapter WholeStageCodegen (59) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] #17 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #17 InputAdapter ReusedExchange [d_date_sk,d_year] #18 InputAdapter @@ -353,12 +353,12 @@ WholeStageCodegen (88) WholeStageCodegen (76) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #22 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #22 InputAdapter WholeStageCodegen (80) Sort [ca_address_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #22 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #22 InputAdapter ReusedExchange [ib_income_band_sk] #23 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index ae65d8f33b996..6c52a1ae74662 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -323,7 +323,7 @@ Input [13]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_cdemo_sk#4, s Output [3]: [s_store_sk#40, s_store_name#41, s_zip#42] Batched: true Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip), IsNotNull(s_store_name)] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] ReadSchema: struct (34) ColumnarToRow [codegen id : 6] @@ -331,7 +331,7 @@ Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] (35) Filter [codegen id : 6] Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] -Condition : ((isnotnull(s_store_sk#40) AND isnotnull(s_zip#42)) AND isnotnull(s_store_name#41)) +Condition : ((isnotnull(s_store_sk#40) AND isnotnull(s_store_name#41)) AND isnotnull(s_zip#42)) (36) BroadcastExchange Input [3]: [s_store_sk#40, s_store_name#41, s_zip#42] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 45e392724d3c1..b6b873868f8cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -1,53 +1,53 @@ WholeStageCodegen (43) - Sort [cnt,product_name,s1,s1,store_name] + Sort [product_name,store_name,cnt,s1,s1] InputAdapter - Exchange [cnt,product_name,s1,s1,store_name] #1 + Exchange [product_name,store_name,cnt,s1,s1] #1 WholeStageCodegen (42) - Project [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,cnt,product_name,s1,s1,s2,s2,s3,s3,store_name,store_zip,syear,syear] - BroadcastHashJoin [cnt,cnt,item_sk,item_sk,store_name,store_name,store_zip,store_zip] - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [b_city,b_streen_name,b_street_number,b_zip,c_city,c_street_name,c_street_number,c_zip,cnt,count,count(1),item_sk,product_name,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + BroadcastHashJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #2 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #2 WholeStageCodegen (20) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -59,18 +59,18 @@ WholeStageCodegen (43) BroadcastExchange #4 WholeStageCodegen (4) Project [cs_item_sk] - Filter [sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price)] - HashAggregate [cs_item_sk,isEmpty,sum,sum] [isEmpty,sum,sum,sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(UnscaledValue(cs_ext_list_price)),sum(cs_ext_list_price)] + Filter [sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true))] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum(cs_ext_list_price),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #5 WholeStageCodegen (3) - HashAggregate [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] [isEmpty,isEmpty,sum,sum,sum,sum] - Project [cr_refunded_cash,cr_reversed_charge,cr_store_credit,cs_ext_list_price,cs_item_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] Filter [cs_item_sk,cs_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_list_price,cs_item_sk,cs_order_number] + Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -81,24 +81,24 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - Filter [s_store_name,s_store_sk,s_zip] + Filter [s_store_sk,s_store_name,s_zip] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_name,s_store_sk,s_zip] + Scan parquet default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) - Filter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Filter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] + Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) @@ -139,9 +139,9 @@ WholeStageCodegen (43) Filter [ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 InputAdapter BroadcastExchange #15 WholeStageCodegen (17) @@ -155,56 +155,56 @@ WholeStageCodegen (43) BroadcastExchange #16 WholeStageCodegen (19) Project [i_item_sk,i_product_name] - Filter [i_color,i_current_price,i_item_sk] + Filter [i_current_price,i_color,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_color,i_current_price,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_current_price,i_color,i_product_name] InputAdapter BroadcastExchange #17 WholeStageCodegen (41) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,count,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,sum,sum,sum] [cnt,count,count(1),item_sk,s1,s2,s3,store_name,store_zip,sum,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_wholesale_cost)),syear] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] InputAdapter - Exchange [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip] #18 + Exchange [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year] #18 WholeStageCodegen (40) - HashAggregate [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] [count,count,sum,sum,sum,sum,sum,sum] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,i_item_sk,i_product_name,s_store_name,s_zip,ss_coupon_amt,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ca_city,ca_city,ca_street_name,ca_street_name,ca_street_number,ca_street_number,ca_zip,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_addr_sk,ca_city,ca_street_name,ca_street_number,ca_zip,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [ca_address_sk,ss_addr_sk] - Project [c_current_addr_sk,d_year,d_year,d_year,hd_income_band_sk,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,hd_income_band_sk,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [hd_demo_sk,ss_hdemo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_wholesale_cost] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [c_current_addr_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,cd_marital_status,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [cd_demo_sk,ss_cdemo_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,d_year,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_shipto_date_sk,d_year,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_first_sales_date_sk,c_first_shipto_date_sk,d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [d_year,s_store_name,s_zip,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_wholesale_cost] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_year,ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Project [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_wholesale_cost] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_addr_sk,ss_cdemo_sk,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,cs_item_sk] + Project [ss_sold_date_sk,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] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Filter [ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_addr_sk,ss_cdemo_sk,ss_coupon_amt,ss_customer_sk,ss_hdemo_sk,ss_item_sk,ss_list_price,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,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] InputAdapter ReusedExchange [sr_item_sk,sr_ticket_number] #3 InputAdapter @@ -212,14 +212,14 @@ WholeStageCodegen (43) InputAdapter BroadcastExchange #19 WholeStageCodegen (25) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [s_store_name,s_store_sk,s_zip] #8 + ReusedExchange [s_store_sk,s_store_name,s_zip] #8 InputAdapter - ReusedExchange [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk] #9 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #10 InputAdapter @@ -235,9 +235,9 @@ WholeStageCodegen (43) InputAdapter ReusedExchange [hd_demo_sk,hd_income_band_sk] #13 InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 InputAdapter - ReusedExchange [ca_address_sk,ca_city,ca_street_name,ca_street_number,ca_zip] #14 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #14 InputAdapter ReusedExchange [ib_income_band_sk] #15 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt index cad2ffb83d0bc..277f3a8edd022 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt @@ -1,50 +1,50 @@ -TakeOrderedAndProject [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,rk,s_store_id,sumsales] +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] WholeStageCodegen (82) Filter [rk] InputAdapter - Window [i_category,sumsales] + Window [sumsales,i_category] WholeStageCodegen (81) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 Union WholeStageCodegen (8) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - Exchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id] #2 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (7) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (4) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #3 WholeStageCodegen (3) - Project [d_moy,d_qoy,d_year,s_store_id,ss_item_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_qoy,d_year,ss_item_sk,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Project [d_date_sk,d_moy,d_qoy,d_year] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_year,d_moy,d_qoy] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter WholeStageCodegen (6) Sort [i_item_sk] @@ -54,76 +54,76 @@ TakeOrderedAndProject [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_n Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (17) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,sum] [isEmpty,s_store_id,sum,sum(sumsales),sumsales] + 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] InputAdapter - Exchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name] #7 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 WholeStageCodegen (16) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (26) - HashAggregate [d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,sum] [d_moy,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [d_qoy,d_year,i_brand,i_category,i_class,i_product_name] #8 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 WholeStageCodegen (25) - HashAggregate [d_qoy,d_year,i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (35) - HashAggregate [d_year,i_brand,i_category,i_class,i_product_name,isEmpty,sum] [d_moy,d_qoy,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [d_year,i_brand,i_category,i_class,i_product_name] #9 + Exchange [i_category,i_class,i_brand,i_product_name,d_year] #9 WholeStageCodegen (34) - HashAggregate [d_year,i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (44) - HashAggregate [i_brand,i_category,i_class,i_product_name,isEmpty,sum] [d_moy,d_qoy,d_year,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name] #10 + Exchange [i_category,i_class,i_brand,i_product_name] #10 WholeStageCodegen (43) - HashAggregate [i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (53) - HashAggregate [i_brand,i_category,i_class,isEmpty,sum] [d_moy,d_qoy,d_year,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_brand,i_category,i_class] #11 + Exchange [i_category,i_class,i_brand] #11 WholeStageCodegen (52) - HashAggregate [i_brand,i_category,i_class,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (62) - HashAggregate [i_category,i_class,isEmpty,sum] [d_moy,d_qoy,d_year,i_brand,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class] #12 WholeStageCodegen (61) - HashAggregate [i_category,i_class,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (71) - HashAggregate [i_category,isEmpty,sum] [d_moy,d_qoy,d_year,i_brand,i_class,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category] #13 WholeStageCodegen (70) - HashAggregate [i_category,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (80) - HashAggregate [isEmpty,sum] [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange #14 WholeStageCodegen (79) - HashAggregate [sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 9fea5d8e164b3..3581b5b7b436c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -1,120 +1,120 @@ -TakeOrderedAndProject [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,rk,s_store_id,sumsales] +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] WholeStageCodegen (55) Filter [rk] InputAdapter - Window [i_category,sumsales] + Window [sumsales,i_category] WholeStageCodegen (54) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 Union WholeStageCodegen (5) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - Exchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id] #2 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (4) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] [isEmpty,isEmpty,sum,sum] - Project [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,s_store_id,ss_quantity,ss_sales_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [d_moy,d_qoy,d_year,s_store_id,ss_item_sk,ss_quantity,ss_sales_price] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [d_moy,d_qoy,d_year,ss_item_sk,ss_quantity,ss_sales_price,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_item_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date_sk,d_moy,d_qoy,d_year] - Filter [d_date_sk,d_month_seq] + Project [d_date_sk,d_year,d_moy,d_qoy] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq,d_moy,d_qoy,d_year] + Scan parquet default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand,i_category,i_class,i_item_sk,i_product_name] + Scan parquet default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] WholeStageCodegen (11) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,sum] [isEmpty,s_store_id,sum,sum(sumsales),sumsales] + 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] InputAdapter - Exchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name] #6 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 WholeStageCodegen (10) - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (17) - HashAggregate [d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,sum] [d_moy,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [d_qoy,d_year,i_brand,i_category,i_class,i_product_name] #7 + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 WholeStageCodegen (16) - HashAggregate [d_qoy,d_year,i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (23) - HashAggregate [d_year,i_brand,i_category,i_class,i_product_name,isEmpty,sum] [d_moy,d_qoy,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [d_year,i_brand,i_category,i_class,i_product_name] #8 + Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8 WholeStageCodegen (22) - HashAggregate [d_year,i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (29) - HashAggregate [i_brand,i_category,i_class,i_product_name,isEmpty,sum] [d_moy,d_qoy,d_year,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_brand,i_category,i_class,i_product_name] #9 + Exchange [i_category,i_class,i_brand,i_product_name] #9 WholeStageCodegen (28) - HashAggregate [i_brand,i_category,i_class,i_product_name,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (35) - HashAggregate [i_brand,i_category,i_class,isEmpty,sum] [d_moy,d_qoy,d_year,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_brand,i_category,i_class] #10 + Exchange [i_category,i_class,i_brand] #10 WholeStageCodegen (34) - HashAggregate [i_brand,i_category,i_class,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (41) - HashAggregate [i_category,i_class,isEmpty,sum] [d_moy,d_qoy,d_year,i_brand,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class] #11 WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (47) - HashAggregate [i_category,isEmpty,sum] [d_moy,d_qoy,d_year,i_brand,i_class,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category] #12 WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 WholeStageCodegen (53) - HashAggregate [isEmpty,sum] [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum,sum(sumsales),sumsales] + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange #13 WholeStageCodegen (52) - HashAggregate [sumsales] [isEmpty,isEmpty,sum,sum] - HashAggregate [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] [isEmpty,sum,sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales] + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00)),sumsales,sum,isEmpty] InputAdapter - ReusedExchange [d_moy,d_qoy,d_year,i_brand,i_category,i_class,i_product_name,isEmpty,s_store_id,sum] #2 + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt index f99d2b02f8429..b3dbc1612539a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_sum] +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] WholeStageCodegen (38) - Project [lochierarchy,rank_within_parent,s_county,s_state,total_sum] + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter - Window [_w0,lochierarchy,total_sum] + Window [total_sum,lochierarchy,_w0] WholeStageCodegen (37) - Sort [_w0,lochierarchy,total_sum] + Sort [lochierarchy,_w0,total_sum] InputAdapter - Exchange [_w0,lochierarchy] #1 + Exchange [lochierarchy,_w0] #1 WholeStageCodegen (36) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] [_w0] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] InputAdapter - Exchange [g_county,g_state,lochierarchy,s_county,s_state,total_sum] #2 + Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 WholeStageCodegen (35) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union WholeStageCodegen (23) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter - Exchange [g_county,g_state,lochierarchy,s_county,s_state,total_sum] #3 + Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #3 WholeStageCodegen (22) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union WholeStageCodegen (10) - HashAggregate [s_county,s_state,sum] [g_county,g_state,lochierarchy,sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] InputAdapter - Exchange [s_county,s_state] #4 + Exchange [s_state,s_county] #4 WholeStageCodegen (9) - HashAggregate [s_county,s_state,ss_net_profit] [sum,sum] - Project [s_county,s_state,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -52,7 +52,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) @@ -61,23 +61,23 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su InputAdapter Window [_w2,s_state] WholeStageCodegen (6) - Sort [_w2,s_state] + Sort [s_state,_w2] InputAdapter Exchange [s_state] #8 WholeStageCodegen (5) - HashAggregate [s_state,sum] [_w2,s_state,sum,sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_state,sum] [sum(UnscaledValue(ss_net_profit)),s_state,_w2,sum] InputAdapter Exchange [s_state] #9 WholeStageCodegen (4) HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [s_state,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Filter [ss_sold_date_sk,ss_store_sk] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -86,22 +86,22 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] WholeStageCodegen (21) - HashAggregate [isEmpty,s_state,sum] [g_county,g_state,isEmpty,lochierarchy,s_county,sum,sum(total_sum),total_sum] + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange [s_state] #11 WholeStageCodegen (20) - HashAggregate [s_state,total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [s_county,s_state,sum] [sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter - ReusedExchange [s_county,s_state,sum] #4 + ReusedExchange [s_state,s_county,sum] #4 WholeStageCodegen (34) - HashAggregate [isEmpty,sum] [g_county,g_state,isEmpty,lochierarchy,s_county,s_state,sum,sum(total_sum),total_sum] + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange #12 WholeStageCodegen (33) - HashAggregate [total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [s_county,s_state,sum] [sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter - ReusedExchange [s_county,s_state,sum] #4 + ReusedExchange [s_state,s_county,sum] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 1cf0fbbea7ac6..bd0bd7e87251f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_sum] +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] WholeStageCodegen (38) - Project [lochierarchy,rank_within_parent,s_county,s_state,total_sum] + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter - Window [_w0,lochierarchy,total_sum] + Window [total_sum,lochierarchy,_w0] WholeStageCodegen (37) - Sort [_w0,lochierarchy,total_sum] + Sort [lochierarchy,_w0,total_sum] InputAdapter - Exchange [_w0,lochierarchy] #1 + Exchange [lochierarchy,_w0] #1 WholeStageCodegen (36) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] [_w0] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] InputAdapter - Exchange [g_county,g_state,lochierarchy,s_county,s_state,total_sum] #2 + Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 WholeStageCodegen (35) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union WholeStageCodegen (23) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter - Exchange [g_county,g_state,lochierarchy,s_county,s_state,total_sum] #3 + Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #3 WholeStageCodegen (22) - HashAggregate [g_county,g_state,lochierarchy,s_county,s_state,total_sum] + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union WholeStageCodegen (10) - HashAggregate [s_county,s_state,sum] [g_county,g_state,lochierarchy,sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] InputAdapter - Exchange [s_county,s_state] #4 + Exchange [s_state,s_county] #4 WholeStageCodegen (9) - HashAggregate [s_county,s_state,ss_net_profit] [sum,sum] - Project [s_county,s_state,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -52,7 +52,7 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_county,s_state] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) @@ -61,47 +61,47 @@ TakeOrderedAndProject [lochierarchy,rank_within_parent,s_county,s_state,total_su InputAdapter Window [_w2,s_state] WholeStageCodegen (6) - Sort [_w2,s_state] + Sort [s_state,_w2] InputAdapter Exchange [s_state] #8 WholeStageCodegen (5) - HashAggregate [s_state,sum] [_w2,s_state,sum,sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_state,sum] [sum(UnscaledValue(ss_net_profit)),s_state,_w2,sum] InputAdapter Exchange [s_state] #9 WholeStageCodegen (4) HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [s_state,ss_net_profit] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [s_state,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Filter [ss_sold_date_sk,ss_store_sk] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_net_profit,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_net_profit] InputAdapter BroadcastExchange #10 WholeStageCodegen (2) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_state,s_store_sk] + Scan parquet default.store [s_store_sk,s_state] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (21) - HashAggregate [isEmpty,s_state,sum] [g_county,g_state,isEmpty,lochierarchy,s_county,sum,sum(total_sum),total_sum] + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange [s_state] #11 WholeStageCodegen (20) - HashAggregate [s_state,total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [s_county,s_state,sum] [sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter - ReusedExchange [s_county,s_state,sum] #4 + ReusedExchange [s_state,s_county,sum] #4 WholeStageCodegen (34) - HashAggregate [isEmpty,sum] [g_county,g_state,isEmpty,lochierarchy,s_county,s_state,sum,sum(total_sum),total_sum] + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter Exchange #12 WholeStageCodegen (33) - HashAggregate [total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [s_county,s_state,sum] [sum,sum(UnscaledValue(ss_net_profit)),total_sum] + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter - ReusedExchange [s_county,s_state,sum] #4 + ReusedExchange [s_state,s_county,sum] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index 1e10cb8da5d09..c2627bd7e4cc9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -160,7 +160,7 @@ Input [8]: [cs_sold_date_sk#4, cs_ship_date_sk#5, cs_bill_cdemo_sk#6, cs_item_sk Output [2]: [d_date_sk#15, d_date#16] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -168,7 +168,7 @@ Input [2]: [d_date_sk#15, d_date#16] (20) Filter [codegen id : 3] Input [2]: [d_date_sk#15, d_date#16] -Condition : (isnotnull(d_date_sk#15) AND isnotnull(d_date#16)) +Condition : (isnotnull(d_date#16) AND isnotnull(d_date_sk#15)) (21) BroadcastExchange Input [2]: [d_date_sk#15, d_date#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index d0b53caa76851..39dba3af02359 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -1,31 +1,31 @@ -TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehouse_name] +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] WholeStageCodegen (21) - HashAggregate [count,d_week_seq,i_item_desc,w_warehouse_name] [count,count(1),no_promo,promo,total_cnt] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter - Exchange [d_week_seq,i_item_desc,w_warehouse_name] #1 + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 WholeStageCodegen (20) - HashAggregate [d_week_seq,i_item_desc,w_warehouse_name] [count,count] - Project [d_week_seq,i_item_desc,w_warehouse_name] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] WholeStageCodegen (17) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #2 WholeStageCodegen (16) - Project [cs_item_sk,cs_order_number,d_week_seq,i_item_desc,w_warehouse_name] + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,d_week_seq,i_item_desc,w_warehouse_name] - SortMergeJoin [cs_item_sk,cs_quantity,d_date_sk,inv_date_sk,inv_item_sk,inv_quantity_on_hand] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter WholeStageCodegen (11) Sort [cs_item_sk,d_date_sk] InputAdapter Exchange [cs_item_sk,d_date_sk] #3 WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,d_date_sk,d_week_seq,i_item_desc] - BroadcastHashJoin [cs_sold_date_sk,d_date,d_date,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_sold_date_sk,d_date,i_item_desc] + Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_date_sk,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] SortMergeJoin [cs_item_sk,i_item_sk] InputAdapter WholeStageCodegen (5) @@ -33,12 +33,12 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (4) - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_sold_date_sk,d_date] + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Project [cs_bill_cdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [hd_demo_sk,cs_bill_hdemo_sk] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -46,16 +46,16 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_marital_status] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] @@ -65,7 +65,7 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (7) Sort [i_item_sk] @@ -75,43 +75,43 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Project [d_date,d_date_sk,d_date_sk,d_week_seq] + Project [d_date_sk,d_date_sk,d_date,d_week_seq] BroadcastHashJoin [d_week_seq,d_week_seq] - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) - Project [d_date,d_date_sk,d_week_seq] - Filter [d_date,d_date_sk,d_week_seq,d_year] + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_week_seq,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter WholeStageCodegen (14) - Sort [inv_date_sk,inv_item_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_date_sk,inv_item_sk] #11 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [w_warehouse_name,inv_date_sk,inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [w_warehouse_sk,inv_warehouse_sk] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #13 WholeStageCodegen (15) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 0c5c4aecec1b4..b66a1f02f7470 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -290,7 +290,7 @@ Input [11]: [cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, Output [2]: [d_date_sk#34, d_date#35] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_date)] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 8] @@ -298,7 +298,7 @@ Input [2]: [d_date_sk#34, d_date#35] (51) Filter [codegen id : 8] Input [2]: [d_date_sk#34, d_date#35] -Condition : (isnotnull(d_date_sk#34) AND isnotnull(d_date#35)) +Condition : (isnotnull(d_date#35) AND isnotnull(d_date_sk#34)) (52) BroadcastExchange Input [2]: [d_date_sk#34, d_date#35] @@ -344,7 +344,7 @@ Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#15, Output [2]: [cr_item_sk#39, cr_order_number#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct (62) ColumnarToRow [codegen id : 10] @@ -352,7 +352,7 @@ Input [2]: [cr_item_sk#39, cr_order_number#40] (63) Filter [codegen id : 10] Input [2]: [cr_item_sk#39, cr_order_number#40] -Condition : (isnotnull(cr_order_number#40) AND isnotnull(cr_item_sk#39)) +Condition : (isnotnull(cr_item_sk#39) AND isnotnull(cr_order_number#40)) (64) BroadcastExchange Input [2]: [cr_item_sk#39, cr_order_number#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 1488d52118b82..25f03cbac8d8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -1,60 +1,60 @@ -TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehouse_name] +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] WholeStageCodegen (12) - HashAggregate [count,d_week_seq,i_item_desc,w_warehouse_name] [count,count(1),no_promo,promo,total_cnt] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter - Exchange [d_week_seq,i_item_desc,w_warehouse_name] #1 + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 WholeStageCodegen (11) - HashAggregate [d_week_seq,i_item_desc,w_warehouse_name] [count,count] - Project [d_week_seq,i_item_desc,w_warehouse_name] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_item_sk,cs_order_number,d_week_seq,i_item_desc,w_warehouse_name] + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,d_week_seq,i_item_desc,w_warehouse_name] - BroadcastHashJoin [cs_ship_date_sk,d_date,d_date,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,d_date,d_week_seq,i_item_desc,w_warehouse_name] - BroadcastHashJoin [d_date_sk,d_week_seq,d_week_seq,inv_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,d_date,d_week_seq,i_item_desc,inv_date_sk,w_warehouse_name] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,i_item_desc,inv_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,i_item_desc,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [cd_demo_sk,cs_bill_cdemo_sk] - Project [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,i_item_desc,inv_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_ship_date_sk,cs_sold_date_sk,inv_date_sk,inv_warehouse_sk] - BroadcastHashJoin [cs_item_sk,cs_quantity,inv_item_sk,inv_quantity_on_hand] - Filter [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,inv_warehouse_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_order_number,cs_promo_sk,cs_quantity,cs_ship_date_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] ColumnarToRow InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_quantity_on_hand,inv_warehouse_sk] + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) Filter [w_warehouse_sk] ColumnarToRow InputAdapter - Scan parquet default.warehouse [w_warehouse_name,w_warehouse_sk] + Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_desc,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) Project [cd_demo_sk] - Filter [cd_demo_sk,cd_marital_status] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] @@ -65,19 +65,19 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_buy_potential,hd_demo_sk] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter BroadcastExchange #7 WholeStageCodegen (6) - Project [d_date,d_date_sk,d_week_seq] - Filter [d_date,d_date_sk,d_week_seq,d_year] + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk,d_week_seq,d_year] + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter BroadcastExchange #8 WholeStageCodegen (7) - Filter [d_date_sk,d_week_seq] + Filter [d_week_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_week_seq] @@ -87,7 +87,7 @@ TakeOrderedAndProject [d_week_seq,i_item_desc,no_promo,promo,total_cnt,w_warehou Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt index c35e70d72eb36..4e4bc8b75134a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (35) - Project [customer_first_name,customer_id,customer_last_name] + Project [customer_id,customer_first_name,customer_last_name] SortMergeJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] InputAdapter WholeStageCodegen (26) - Project [customer_first_name,customer_id,customer_id,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] SortMergeJoin [customer_id,customer_id] InputAdapter WholeStageCodegen (17) @@ -16,29 +16,29 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Exchange [customer_id] #1 WholeStageCodegen (7) Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #2 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #3 WholeStageCodegen (2) - Project [d_year,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -48,39 +48,39 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] InputAdapter Exchange [c_customer_sk] #5 WholeStageCodegen (4) - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter WholeStageCodegen (16) Sort [customer_id] InputAdapter Exchange [customer_id] #6 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_first_name,customer_id,customer_last_name,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #7 WholeStageCodegen (14) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - SortMergeJoin [c_customer_sk,ss_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (11) Sort [ss_customer_sk] InputAdapter Exchange [ss_customer_sk] #8 WholeStageCodegen (10) - Project [d_year,ss_customer_sk,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_customer_sk,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] InputAdapter BroadcastExchange #9 WholeStageCodegen (9) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -88,7 +88,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (13) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_id,c_customer_sk,c_first_name,c_last_name] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 InputAdapter WholeStageCodegen (25) Sort [customer_id] @@ -97,61 +97,61 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (24) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 WholeStageCodegen (23) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (20) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #12 WholeStageCodegen (19) - Project [d_year,ws_bill_customer_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] InputAdapter ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_id,c_customer_sk,c_first_name,c_last_name] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 InputAdapter WholeStageCodegen (34) Sort [customer_id] InputAdapter Exchange [customer_id] #13 WholeStageCodegen (33) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 WholeStageCodegen (32) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - SortMergeJoin [c_customer_sk,ws_bill_customer_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (29) Sort [ws_bill_customer_sk] InputAdapter Exchange [ws_bill_customer_sk] #15 WholeStageCodegen (28) - Project [d_year,ws_bill_customer_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_bill_customer_sk,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] InputAdapter ReusedExchange [d_date_sk,d_year] #9 InputAdapter WholeStageCodegen (31) Sort [c_customer_sk] InputAdapter - ReusedExchange [c_customer_id,c_customer_sk,c_first_name,c_last_name] #5 + ReusedExchange [c_customer_sk,c_customer_id,c_first_name,c_last_name] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 68a6e7bfd91a3..205fb6ac974a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -1,60 +1,60 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (16) - Project [customer_first_name,customer_id,customer_last_name] + Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_first_name,customer_id,customer_id,customer_last_name,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] BroadcastHashJoin [customer_id,customer_id] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 WholeStageCodegen (3) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_sold_date_sk,ss_net_paid] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_net_paid] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_first_name,customer_id,customer_last_name,sum,sum(UnscaledValue(ss_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_sold_date_sk,ss_net_paid] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + ReusedExchange [ss_sold_date_sk,ss_customer_sk,ss_net_paid] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -63,45 +63,45 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] WholeStageCodegen (11) Project [customer_id,year_total] Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #8 WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_sold_date_sk,ws_net_paid] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) Filter [ws_bill_customer_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #10 WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [customer_id,sum,sum(UnscaledValue(ws_net_paid)),year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 WholeStageCodegen (14) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_sold_date_sk,ws_net_paid] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - Filter [c_customer_id,c_customer_sk] + Filter [c_customer_sk,c_customer_id] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_id,c_customer_sk,c_first_name,c_last_name] + Scan parquet default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - ReusedExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #9 + ReusedExchange [ws_sold_date_sk,ws_bill_customer_sk,ws_net_paid] #9 InputAdapter ReusedExchange [d_date_sk,d_year] #6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt index f947a62f173a6..3f452dc9272dc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt @@ -157,7 +157,7 @@ Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_class_id), IsNotNull(i_brand_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -165,7 +165,7 @@ Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category (6) Filter [codegen id : 1] Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] -Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_class_id#8)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] @@ -223,7 +223,7 @@ Arguments: [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST], fa Output [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 5] @@ -231,7 +231,7 @@ Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_ (21) Filter [codegen id : 5] Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] -Condition : (isnotnull(cr_item_sk#17) AND isnotnull(cr_order_number#18)) +Condition : (isnotnull(cr_order_number#18) AND isnotnull(cr_item_sk#17)) (22) Exchange Input [4]: [cr_item_sk#17, cr_order_number#18, cr_return_quantity#19, cr_return_amount#20] @@ -300,7 +300,7 @@ Arguments: [cast(ss_ticket_number#26 as bigint) ASC NULLS FIRST, cast(ss_item_sk Output [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 12] @@ -308,7 +308,7 @@ Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return (39) Filter [codegen id : 12] Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] -Condition : (isnotnull(sr_item_sk#30) AND isnotnull(sr_ticket_number#31)) +Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (40) Exchange Input [4]: [sr_item_sk#30, sr_ticket_number#31, sr_return_quantity#32, sr_return_amt#33] @@ -397,7 +397,7 @@ Arguments: [cast(ws_order_number#40 as bigint) ASC NULLS FIRST, cast(ws_item_sk# Output [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 21] @@ -405,7 +405,7 @@ Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_ (61) Filter [codegen id : 21] Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] -Condition : (isnotnull(wr_item_sk#44) AND isnotnull(wr_order_number#45)) +Condition : (isnotnull(wr_order_number#45) AND isnotnull(wr_item_sk#44)) (62) Exchange Input [4]: [wr_item_sk#44, wr_order_number#45, wr_return_quantity#46, wr_return_amt#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt index 40aa2931ad5b8..69f8b6a5b6789 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt @@ -1,237 +1,237 @@ -TakeOrderedAndProject [curr_yr_cnt,i_brand_id,i_category_id,i_class_id,i_manufact_id,prev_year,prev_yr_cnt,sales_amt_diff,sales_cnt_diff,year] +TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] WholeStageCodegen (55) - Project [d_year,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_amt,sales_cnt,sales_cnt] - SortMergeJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,i_manufact_id,i_manufact_id,sales_cnt,sales_cnt] + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter WholeStageCodegen (27) - Sort [i_brand_id,i_category_id,i_class_id,i_manufact_id] + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id,i_manufact_id] #1 + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 WholeStageCodegen (26) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #2 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 WholeStageCodegen (25) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #3 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 WholeStageCodegen (24) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (16) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #4 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #4 WholeStageCodegen (15) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (7) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (4) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #5 + Exchange [cs_order_number,cs_item_sk] #5 WholeStageCodegen (3) - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (1) - Project [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] - Filter [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] InputAdapter BroadcastExchange #7 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (6) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #8 + Exchange [cr_order_number,cr_item_sk] #8 WholeStageCodegen (5) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] WholeStageCodegen (14) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] + 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] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] WholeStageCodegen (11) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #9 + Exchange [ss_ticket_number,ss_item_sk] #9 WholeStageCodegen (10) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #7 WholeStageCodegen (13) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #10 + Exchange [sr_ticket_number,sr_item_sk] #10 WholeStageCodegen (12) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] WholeStageCodegen (23) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] + 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] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] WholeStageCodegen (20) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #11 + Exchange [ws_order_number,ws_item_sk] #11 WholeStageCodegen (19) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #7 WholeStageCodegen (22) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_item_sk,wr_order_number] #12 + Exchange [wr_order_number,wr_item_sk] #12 WholeStageCodegen (21) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] InputAdapter WholeStageCodegen (54) - Sort [i_brand_id,i_category_id,i_class_id,i_manufact_id] + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_category_id,i_class_id,i_manufact_id] #13 + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 WholeStageCodegen (53) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #14 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 WholeStageCodegen (52) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #15 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 WholeStageCodegen (51) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (43) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #16 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #16 WholeStageCodegen (42) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (34) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (31) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #17 + Exchange [cs_order_number,cs_item_sk] #17 WholeStageCodegen (30) - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter BroadcastExchange #18 WholeStageCodegen (29) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] WholeStageCodegen (33) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] #8 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 WholeStageCodegen (41) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] + 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] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] WholeStageCodegen (38) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #19 + Exchange [ss_ticket_number,ss_item_sk] #19 WholeStageCodegen (37) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #18 WholeStageCodegen (40) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] #10 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 WholeStageCodegen (50) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] + 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] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] WholeStageCodegen (47) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #20 + Exchange [ws_order_number,ws_item_sk] #20 WholeStageCodegen (46) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 InputAdapter ReusedExchange [d_date_sk,d_year] #18 WholeStageCodegen (49) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] #12 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 1653191ed3ab8..1d8aab417f188 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -136,7 +136,7 @@ Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) Output [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_category_id), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -144,7 +144,7 @@ Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category (6) Filter [codegen id : 1] Input [6]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_category#10, i_manufact_id#11] -Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_category_id#9)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_manufact_id#11)) +Condition : ((((((isnotnull(i_category#10) AND (i_category#10 = Books)) AND isnotnull(i_item_sk#6)) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) AND isnotnull(i_manufact_id#11)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, i_manufact_id#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index 4974c17705d87..d1c20801ec5fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -1,180 +1,180 @@ -TakeOrderedAndProject [curr_yr_cnt,i_brand_id,i_category_id,i_class_id,i_manufact_id,prev_year,prev_yr_cnt,sales_amt_diff,sales_cnt_diff,year] +TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] WholeStageCodegen (34) - Project [d_year,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_amt,sales_cnt,sales_cnt] - BroadcastHashJoin [i_brand_id,i_brand_id,i_category_id,i_category_id,i_class_id,i_class_id,i_manufact_id,i_manufact_id,sales_cnt,sales_cnt] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #1 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 WholeStageCodegen (16) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #2 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #2 WholeStageCodegen (15) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (10) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #3 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 WholeStageCodegen (9) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (4) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) - Project [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] - Filter [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] ColumnarToRow InputAdapter - Scan parquet default.item [i_brand_id,i_category,i_category_id,i_class_id,i_item_sk,i_manufact_id] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #6 WholeStageCodegen (3) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] WholeStageCodegen (8) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + 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] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] WholeStageCodegen (14) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + 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] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #5 InputAdapter BroadcastExchange #8 WholeStageCodegen (13) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] InputAdapter BroadcastExchange #9 WholeStageCodegen (33) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sum,sum] [sales_amt,sales_cnt,sum,sum,sum(UnscaledValue(sales_amt)),sum(cast(sales_cnt as bigint))] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(cast(sales_cnt as bigint)),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] #10 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #10 WholeStageCodegen (32) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] [sum,sum,sum,sum] - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #11 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #11 WholeStageCodegen (31) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (26) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] #12 + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #12 WholeStageCodegen (25) - HashAggregate [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sales_amt,sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union WholeStageCodegen (20) - Project [cr_return_amount,cr_return_quantity,cs_ext_sales_price,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Project [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] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk,i_brand_id,i_category_id,i_class_id,i_manufact_id] + Project [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_ext_sales_price,cs_item_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter BroadcastExchange #13 WholeStageCodegen (18) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_amount,cr_return_quantity] #6 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #6 WholeStageCodegen (24) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,sr_return_amt,sr_return_quantity,ss_ext_sales_price,ss_quantity] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_ticket_number] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] - BroadcastHashJoin [i_item_sk,ss_item_sk] + 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] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Project [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] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_quantity,ss_sold_date_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #13 InputAdapter - ReusedExchange [sr_item_sk,sr_return_amt,sr_return_quantity,sr_ticket_number] #7 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #7 WholeStageCodegen (30) - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,wr_return_amt,wr_return_quantity,ws_ext_sales_price,ws_quantity] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Project [d_year,i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [i_brand_id,i_category_id,i_class_id,i_manufact_id,ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] + 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] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Project [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] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_order_number,ws_quantity,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price] InputAdapter - ReusedExchange [i_brand_id,i_category_id,i_class_id,i_item_sk,i_manufact_id] #4 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #4 InputAdapter ReusedExchange [d_date_sk,d_year] #13 InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_return_quantity] #8 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt index 2155d4546ab99..92c25891f940e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/simplified.txt @@ -1,44 +1,44 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (81) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #1 + Exchange [channel,id,sales,returns,profit] #1 WholeStageCodegen (80) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (53) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #2 + Exchange [channel,id,sales,returns,profit] #2 WholeStageCodegen (52) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (25) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #3 WholeStageCodegen (24) - HashAggregate [channel,id,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (8) - Project [profit,profit_loss,returns,s_store_sk,sales] + Project [s_store_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #4 WholeStageCodegen (3) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [s_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -46,7 +46,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -57,29 +57,29 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(sr_return_amt))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [s_store_sk] #8 WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_net_loss,sr_return_amt] [sum,sum,sum,sum] - Project [s_store_sk,sr_net_loss,sr_return_amt] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] - Project [s_store_sk,sr_net_loss,sr_return_amt,sr_returned_date_sk] - BroadcastHashJoin [s_store_sk,sr_store_sk] + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [sr_returned_date_sk,sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter ReusedExchange [s_store_sk] #6 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (15) - Project [cs_call_center_sk,profit,profit_loss,returns,sales] + Project [cs_call_center_sk,sales,returns,profit,profit_loss] InputAdapter BroadcastNestedLoopJoin WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [cs_call_center_sk] #9 WholeStageCodegen (10) @@ -89,40 +89,40 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_ext_sales_price,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 BroadcastExchange #10 WholeStageCodegen (14) - HashAggregate [sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(cr_net_loss)),sum(UnscaledValue(cr_return_amount))] + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange #11 WholeStageCodegen (13) - HashAggregate [cr_net_loss,cr_return_amount] [sum,sum,sum,sum] - Project [cr_net_loss,cr_return_amount] + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (23) - Project [profit,profit_loss,returns,sales,wp_web_page_sk] + Project [wp_web_page_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [sum,sum,wp_web_page_sk] [profit,sales,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [wp_web_page_sk] #12 WholeStageCodegen (18) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] - Project [ws_ext_sales_price,ws_net_profit,ws_web_page_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_page_sk,ws_ext_sales_price,ws_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -135,38 +135,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #14 WholeStageCodegen (22) - HashAggregate [sum,sum,wp_web_page_sk] [profit_loss,returns,sum,sum,sum(UnscaledValue(wr_net_loss)),sum(UnscaledValue(wr_return_amt))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [wp_web_page_sk] #15 WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,wr_net_loss,wr_return_amt] [sum,sum,sum,sum] - Project [wp_web_page_sk,wr_net_loss,wr_return_amt] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] - Project [wp_web_page_sk,wr_net_loss,wr_return_amt,wr_returned_date_sk] - BroadcastHashJoin [wp_web_page_sk,wr_web_page_sk] + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Project [wr_returned_date_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] Filter [wr_returned_date_sk,wr_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_net_loss,wr_return_amt,wr_returned_date_sk,wr_web_page_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_web_page_sk,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [wp_web_page_sk] #13 InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (51) - HashAggregate [channel,isEmpty,isEmpty,isEmpty,sum,sum,sum] [id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #16 WholeStageCodegen (50) - HashAggregate [channel,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 WholeStageCodegen (79) - HashAggregate [isEmpty,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #17 WholeStageCodegen (78) - HashAggregate [profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index a264d4273b546..864039e512231 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -1,44 +1,44 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (81) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #1 + Exchange [channel,id,sales,returns,profit] #1 WholeStageCodegen (80) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (53) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #2 + Exchange [channel,id,sales,returns,profit] #2 WholeStageCodegen (52) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (25) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #3 WholeStageCodegen (24) - HashAggregate [channel,id,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (8) - Project [profit,profit_loss,returns,s_store_sk,sales] + Project [s_store_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #4 WholeStageCodegen (3) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [s_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,ss_store_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -46,7 +46,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -57,30 +57,30 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #7 WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(sr_return_amt))] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [s_store_sk] #8 WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_net_loss,sr_return_amt] [sum,sum,sum,sum] - Project [s_store_sk,sr_net_loss,sr_return_amt] - BroadcastHashJoin [s_store_sk,sr_store_sk] - Project [sr_net_loss,sr_return_amt,sr_store_sk] - BroadcastHashJoin [d_date_sk,sr_returned_date_sk] + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] Filter [sr_returned_date_sk,sr_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_net_loss,sr_return_amt,sr_returned_date_sk,sr_store_sk] + Scan parquet default.store_returns [sr_returned_date_sk,sr_store_sk,sr_return_amt,sr_net_loss] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [s_store_sk] #6 WholeStageCodegen (15) - Project [cs_call_center_sk,profit,profit_loss,returns,sales] + Project [cs_call_center_sk,sales,returns,profit,profit_loss] InputAdapter BroadcastNestedLoopJoin BroadcastExchange #9 WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [profit,sales,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [cs_call_center_sk] #10 WholeStageCodegen (10) @@ -90,39 +90,39 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cs_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_call_center_sk,cs_ext_sales_price,cs_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (14) - HashAggregate [sum,sum] [profit_loss,returns,sum,sum,sum(UnscaledValue(cr_net_loss)),sum(UnscaledValue(cr_return_amount))] + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange #11 WholeStageCodegen (13) - HashAggregate [cr_net_loss,cr_return_amount] [sum,sum,sum,sum] - Project [cr_net_loss,cr_return_amount] + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_net_loss,cr_return_amount,cr_returned_date_sk] + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #5 WholeStageCodegen (23) - Project [profit,profit_loss,returns,sales,wp_web_page_sk] + Project [wp_web_page_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [sum,sum,wp_web_page_sk] [profit,sales,sum,sum,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [wp_web_page_sk] #12 WholeStageCodegen (18) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [wp_web_page_sk,ws_web_page_sk] - Project [ws_ext_sales_price,ws_net_profit,ws_web_page_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,ws_web_page_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_web_page_sk,ws_ext_sales_price,ws_net_profit] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter @@ -135,38 +135,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] InputAdapter BroadcastExchange #14 WholeStageCodegen (22) - HashAggregate [sum,sum,wp_web_page_sk] [profit_loss,returns,sum,sum,sum(UnscaledValue(wr_net_loss)),sum(UnscaledValue(wr_return_amt))] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter Exchange [wp_web_page_sk] #15 WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,wr_net_loss,wr_return_amt] [sum,sum,sum,sum] - Project [wp_web_page_sk,wr_net_loss,wr_return_amt] - BroadcastHashJoin [wp_web_page_sk,wr_web_page_sk] - Project [wr_net_loss,wr_return_amt,wr_web_page_sk] - BroadcastHashJoin [d_date_sk,wr_returned_date_sk] + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] Filter [wr_returned_date_sk,wr_web_page_sk] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_net_loss,wr_return_amt,wr_returned_date_sk,wr_web_page_sk] + Scan parquet default.web_returns [wr_returned_date_sk,wr_web_page_sk,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [wp_web_page_sk] #13 WholeStageCodegen (51) - HashAggregate [channel,isEmpty,isEmpty,isEmpty,sum,sum,sum] [id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #16 WholeStageCodegen (50) - HashAggregate [channel,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 WholeStageCodegen (79) - HashAggregate [isEmpty,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #17 WholeStageCodegen (78) - HashAggregate [profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index 9c328f4d3d49a..fc9634469d6df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -75,7 +75,7 @@ TakeOrderedAndProject (70) Output [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -83,7 +83,7 @@ Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number# (3) Filter [codegen id : 1] Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_item_sk#2)) +Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) (4) Exchange Input [7]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#4, ss_quantity#5, ss_wholesale_cost#6, ss_sales_price#7] @@ -285,7 +285,7 @@ Input [12]: [ss_sold_year#25, ss_item_sk#2, ss_customer_sk#3, ss_qty#26, ss_wc#2 Output [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_number#58, ws_quantity#59, ws_wholesale_cost#60, ws_sales_price#61] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 16] @@ -293,7 +293,7 @@ Input [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_ (49) Filter [codegen id : 16] Input [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_number#58, ws_quantity#59, ws_wholesale_cost#60, ws_sales_price#61] -Condition : ((isnotnull(ws_sold_date_sk#55) AND isnotnull(ws_bill_customer_sk#57)) AND isnotnull(ws_item_sk#56)) +Condition : ((isnotnull(ws_sold_date_sk#55) AND isnotnull(ws_item_sk#56)) AND isnotnull(ws_bill_customer_sk#57)) (50) Exchange Input [7]: [ws_sold_date_sk#55, ws_item_sk#56, ws_bill_customer_sk#57, ws_order_number#58, ws_quantity#59, ws_wholesale_cost#60, ws_sales_price#61] @@ -382,10 +382,10 @@ Right keys [3]: [ws_sold_year#76, ws_item_sk#56, ws_customer_sk#77] Join condition: None (69) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#26 as double) / cast(coalesce((ws_qty#78 + cs_qty#52), 1) as double)), 2) AS ratio#81, ss_qty#26 AS store_qty#82, ss_wc#27 AS store_wholesale_cost#83, ss_sp#28 AS store_sales_price#84, (coalesce(ws_qty#78, 0) + coalesce(cs_qty#52, 0)) AS other_chan_qty#85, CheckOverflow((promote_precision(cast(coalesce(ws_wc#79, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#53, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#86, CheckOverflow((promote_precision(cast(coalesce(ws_sp#80, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#54, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#87, ss_sold_year#25, ss_wc#27, ss_customer_sk#3, ss_qty#26, ss_sp#28, ss_item_sk#2] +Output [13]: [round((cast(ss_qty#26 as double) / cast(coalesce((ws_qty#78 + cs_qty#52), 1) as double)), 2) AS ratio#81, ss_qty#26 AS store_qty#82, ss_wc#27 AS store_wholesale_cost#83, ss_sp#28 AS store_sales_price#84, (coalesce(ws_qty#78, 0) + coalesce(cs_qty#52, 0)) AS other_chan_qty#85, CheckOverflow((promote_precision(cast(coalesce(ws_wc#79, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#53, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#86, CheckOverflow((promote_precision(cast(coalesce(ws_sp#80, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#54, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#87, ss_sold_year#25, ss_item_sk#2, ss_customer_sk#3, ss_qty#26, ss_wc#27, ss_sp#28] Input [15]: [ss_sold_year#25, ss_item_sk#2, ss_customer_sk#3, ss_qty#26, ss_wc#27, ss_sp#28, cs_qty#52, cs_wc#53, cs_sp#54, ws_sold_year#76, ws_item_sk#56, ws_customer_sk#77, ws_qty#78, ws_wc#79, ws_sp#80] (70) TakeOrderedAndProject -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#25, ss_wc#27, ss_customer_sk#3, ss_qty#26, ss_sp#28, ss_item_sk#2] +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#25, ss_item_sk#2, ss_customer_sk#3, ss_qty#26, ss_wc#27, ss_sp#28] Arguments: 100, [ss_sold_year#25 ASC NULLS FIRST, ss_item_sk#2 ASC NULLS FIRST, ss_customer_sk#3 ASC NULLS FIRST, ss_qty#26 DESC NULLS LAST, ss_wc#27 DESC NULLS LAST, ss_sp#28 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/simplified.txt index 11d27f3663012..097d27131cb21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/simplified.txt @@ -1,80 +1,80 @@ -TakeOrderedAndProject [other_chan_qty,other_chan_sales_price,other_chan_wholesale_cost,ratio,ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc,store_qty,store_sales_price,store_wholesale_cost] +TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (23) - Project [cs_qty,cs_sp,cs_wc,ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc,ws_qty,ws_sp,ws_wc] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_sold_year,ws_customer_sk,ws_item_sk,ws_sold_year] + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] InputAdapter WholeStageCodegen (15) - Project [cs_qty,cs_sp,cs_wc,ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc] - SortMergeJoin [cs_customer_sk,cs_item_sk,cs_sold_year,ss_customer_sk,ss_item_sk,ss_sold_year] + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,cs_qty,cs_wc,cs_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk,ss_item_sk,ss_sold_year] - HashAggregate [d_year,ss_customer_sk,ss_item_sk,sum,sum,sum] [ss_qty,ss_sold_year,ss_sp,ss_wc,sum,sum,sum,sum(UnscaledValue(ss_sales_price)),sum(UnscaledValue(ss_wholesale_cost)),sum(cast(ss_quantity as bigint))] + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ss_customer_sk,ss_item_sk] #1 + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 WholeStageCodegen (6) - HashAggregate [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_wholesale_cost] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] Filter [sr_ticket_number] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] WholeStageCodegen (2) - Sort [ss_item_sk,ss_ticket_number] + Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_item_sk,ss_ticket_number] #2 + Exchange [ss_ticket_number,ss_item_sk] #2 WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price] WholeStageCodegen (4) - Sort [sr_item_sk,sr_ticket_number] + Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_item_sk,sr_ticket_number] #3 + Exchange [sr_ticket_number,sr_item_sk] #3 WholeStageCodegen (3) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #4 WholeStageCodegen (5) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter WholeStageCodegen (14) - Sort [cs_customer_sk,cs_item_sk,cs_sold_year] + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] Filter [cs_qty] - HashAggregate [cs_bill_customer_sk,cs_item_sk,d_year,sum,sum,sum] [cs_customer_sk,cs_qty,cs_sold_year,cs_sp,cs_wc,sum,sum,sum,sum(UnscaledValue(cs_sales_price)),sum(UnscaledValue(cs_wholesale_cost)),sum(cast(cs_quantity as bigint))] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cast(cs_quantity as bigint)),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk,d_year] #5 + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #5 WholeStageCodegen (13) - HashAggregate [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] Filter [cr_order_number] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] WholeStageCodegen (9) - Sort [cs_item_sk,cs_order_number] + Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_item_sk,cs_order_number] #6 + Exchange [cs_order_number,cs_item_sk] #6 WholeStageCodegen (8) - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_item_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price] WholeStageCodegen (11) - Sort [cr_item_sk,cr_order_number] + Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_item_sk,cr_order_number] #7 + Exchange [cr_order_number,cr_item_sk] #7 WholeStageCodegen (10) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_order_number] @@ -82,34 +82,34 @@ TakeOrderedAndProject [other_chan_qty,other_chan_sales_price,other_chan_wholesal ReusedExchange [d_date_sk,d_year] #4 InputAdapter WholeStageCodegen (22) - Sort [ws_customer_sk,ws_item_sk,ws_sold_year] + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] Filter [ws_qty] - HashAggregate [d_year,sum,sum,sum,ws_bill_customer_sk,ws_item_sk] [sum,sum,sum,sum(UnscaledValue(ws_sales_price)),sum(UnscaledValue(ws_wholesale_cost)),sum(cast(ws_quantity as bigint)),ws_customer_sk,ws_qty,ws_sold_year,ws_sp,ws_wc] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(cast(ws_quantity as bigint)),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ws_bill_customer_sk,ws_item_sk] #8 + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #8 WholeStageCodegen (21) - HashAggregate [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] Filter [wr_order_number] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] WholeStageCodegen (17) - Sort [ws_item_sk,ws_order_number] + Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_item_sk,ws_order_number] #9 + Exchange [ws_order_number,ws_item_sk] #9 WholeStageCodegen (16) - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + Filter [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price] WholeStageCodegen (19) - Sort [wr_item_sk,wr_order_number] + Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_item_sk,wr_order_number] #10 + Exchange [wr_order_number,wr_item_sk] #10 WholeStageCodegen (18) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 11f3971bb6855..747ad5194ef2b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -155,7 +155,7 @@ Results [6]: [d_year#12 AS ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, sum( Output [7]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] @@ -163,7 +163,7 @@ Input [7]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_ (22) Filter [codegen id : 6] Input [7]: [ws_sold_date_sk#28, ws_item_sk#29, ws_bill_customer_sk#30, ws_order_number#31, ws_quantity#32, ws_wholesale_cost#33, ws_sales_price#34] -Condition : ((isnotnull(ws_sold_date_sk#28) AND isnotnull(ws_bill_customer_sk#30)) AND isnotnull(ws_item_sk#29)) +Condition : ((isnotnull(ws_sold_date_sk#28) AND isnotnull(ws_item_sk#29)) AND isnotnull(ws_bill_customer_sk#30)) (23) Scan parquet default.web_returns Output [2]: [wr_item_sk#35, wr_order_number#36] @@ -332,10 +332,10 @@ Right keys [3]: [cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75] Join condition: None (59) Project [codegen id : 12] -Output [13]: [round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) AS ratio#80, ss_qty#25 AS store_qty#81, ss_wc#26 AS store_wholesale_cost#82, ss_sp#27 AS store_sales_price#83, (coalesce(ws_qty#50, 0) + coalesce(cs_qty#76, 0)) AS other_chan_qty#84, CheckOverflow((promote_precision(cast(coalesce(ws_wc#51, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#77, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#85, CheckOverflow((promote_precision(cast(coalesce(ws_sp#52, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#78, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#86, ss_sold_year#24, ss_qty#25, ss_item_sk#2, ss_wc#26, ss_customer_sk#3, ss_sp#27] +Output [13]: [round((cast(ss_qty#25 as double) / cast(coalesce((ws_qty#50 + cs_qty#76), 1) as double)), 2) AS ratio#80, ss_qty#25 AS store_qty#81, ss_wc#26 AS store_wholesale_cost#82, ss_sp#27 AS store_sales_price#83, (coalesce(ws_qty#50, 0) + coalesce(cs_qty#76, 0)) AS other_chan_qty#84, CheckOverflow((promote_precision(cast(coalesce(ws_wc#51, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#77, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#85, CheckOverflow((promote_precision(cast(coalesce(ws_sp#52, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#78, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#86, ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27] Input [15]: [ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27, ws_qty#50, ws_wc#51, ws_sp#52, cs_sold_year#74, cs_item_sk#56, cs_customer_sk#75, cs_qty#76, cs_wc#77, cs_sp#78] (60) TakeOrderedAndProject -Input [13]: [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86, ss_sold_year#24, ss_qty#25, ss_item_sk#2, ss_wc#26, ss_customer_sk#3, ss_sp#27] +Input [13]: [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86, ss_sold_year#24, ss_item_sk#2, ss_customer_sk#3, ss_qty#25, ss_wc#26, ss_sp#27] Arguments: 100, [ss_sold_year#24 ASC NULLS FIRST, ss_item_sk#2 ASC NULLS FIRST, ss_customer_sk#3 ASC NULLS FIRST, ss_qty#25 DESC NULLS LAST, ss_wc#26 DESC NULLS LAST, ss_sp#27 DESC NULLS LAST, other_chan_qty#84 ASC NULLS FIRST, other_chan_wholesale_cost#85 ASC NULLS FIRST, other_chan_sales_price#86 ASC NULLS FIRST, ratio#80 ASC NULLS FIRST], [ratio#80, store_qty#81, store_wholesale_cost#82, store_sales_price#83, other_chan_qty#84, other_chan_wholesale_cost#85, other_chan_sales_price#86] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt index 85ead08da9447..7f5008a59b50d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt @@ -1,34 +1,34 @@ -TakeOrderedAndProject [other_chan_qty,other_chan_sales_price,other_chan_wholesale_cost,ratio,ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc,store_qty,store_sales_price,store_wholesale_cost] +TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (12) - Project [cs_qty,cs_sp,cs_wc,ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc,ws_qty,ws_sp,ws_wc] - BroadcastHashJoin [cs_customer_sk,cs_item_sk,cs_sold_year,ss_customer_sk,ss_item_sk,ss_sold_year] - Project [ss_customer_sk,ss_item_sk,ss_qty,ss_sold_year,ss_sp,ss_wc,ws_qty,ws_sp,ws_wc] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_sold_year,ws_customer_sk,ws_item_sk,ws_sold_year] - HashAggregate [d_year,ss_customer_sk,ss_item_sk,sum,sum,sum] [ss_qty,ss_sold_year,ss_sp,ss_wc,sum,sum,sum,sum(UnscaledValue(ss_sales_price)),sum(UnscaledValue(ss_wholesale_cost)),sum(cast(ss_quantity as bigint))] + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + BroadcastHashJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(cast(ss_quantity as bigint)),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ss_customer_sk,ss_item_sk] #1 + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 WholeStageCodegen (3) - HashAggregate [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_wholesale_cost] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_wholesale_cost] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] Filter [sr_ticket_number] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_customer_sk,ss_item_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,ss_ticket_number,ss_wholesale_cost] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Filter [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_returns [sr_item_sk,sr_ticket_number] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Filter [d_date_sk,d_year] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] @@ -36,24 +36,24 @@ TakeOrderedAndProject [other_chan_qty,other_chan_sales_price,other_chan_wholesal BroadcastExchange #4 WholeStageCodegen (7) Filter [ws_qty] - HashAggregate [d_year,sum,sum,sum,ws_bill_customer_sk,ws_item_sk] [sum,sum,sum,sum(UnscaledValue(ws_sales_price)),sum(UnscaledValue(ws_wholesale_cost)),sum(cast(ws_quantity as bigint)),ws_customer_sk,ws_qty,ws_sold_year,ws_sp,ws_wc] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(cast(ws_quantity as bigint)),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] InputAdapter - Exchange [d_year,ws_bill_customer_sk,ws_item_sk] #5 + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 WholeStageCodegen (6) - HashAggregate [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] [sum,sum,sum,sum,sum,sum] - Project [d_year,ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_wholesale_cost] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [ws_bill_customer_sk,ws_item_sk,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] Filter [wr_order_number] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [ws_bill_customer_sk,ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + Filter [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_bill_customer_sk,ws_item_sk,ws_order_number,ws_quantity,ws_sales_price,ws_sold_date_sk,ws_wholesale_cost] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price] InputAdapter BroadcastExchange #6 WholeStageCodegen (4) - Filter [wr_item_sk,wr_order_number] + Filter [wr_order_number,wr_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_order_number] @@ -63,24 +63,24 @@ TakeOrderedAndProject [other_chan_qty,other_chan_sales_price,other_chan_wholesal BroadcastExchange #7 WholeStageCodegen (11) Filter [cs_qty] - HashAggregate [cs_bill_customer_sk,cs_item_sk,d_year,sum,sum,sum] [cs_customer_sk,cs_qty,cs_sold_year,cs_sp,cs_wc,sum,sum,sum,sum(UnscaledValue(cs_sales_price)),sum(UnscaledValue(cs_wholesale_cost)),sum(cast(cs_quantity as bigint))] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cast(cs_quantity as bigint)),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk,d_year] #8 + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 WholeStageCodegen (10) - HashAggregate [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_wholesale_cost,d_year] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] Filter [cr_order_number] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + Filter [cs_sold_date_sk,cs_item_sk,cs_bill_customer_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_sales_price,cs_sold_date_sk,cs_wholesale_cost] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (8) - Filter [cr_item_sk,cr_order_number] + Filter [cr_order_number,cr_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_order_number] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt index 452668525aeba..13781c8bd5993 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/simplified.txt @@ -1,54 +1,54 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (102) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #1 + Exchange [channel,id,sales,returns,profit] #1 WholeStageCodegen (101) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (67) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #2 + Exchange [channel,id,sales,returns,profit] #2 WholeStageCodegen (66) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (32) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #3 WholeStageCodegen (31) - HashAggregate [channel,id,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (10) - HashAggregate [isEmpty,isEmpty,s_store_id,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [s_store_id] #4 WholeStageCodegen (9) - HashAggregate [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_store_sk] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_promo_sk,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] InputAdapter - SortMergeJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] WholeStageCodegen (2) Sort [ss_item_sk,ss_ticket_number] InputAdapter Exchange [ss_item_sk,ss_ticket_number] #5 WholeStageCodegen (1) - Filter [ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit] WholeStageCodegen (4) Sort [sr_item_sk,sr_ticket_number] InputAdapter @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [sr_item_sk,sr_ticket_number] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_net_loss,sr_return_amt,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) @@ -73,7 +73,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #9 WholeStageCodegen (7) @@ -81,40 +81,40 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_tv,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] InputAdapter BroadcastExchange #10 WholeStageCodegen (8) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] WholeStageCodegen (20) - HashAggregate [cp_catalog_page_id,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00))] + 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(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cp_catalog_page_id] #11 WholeStageCodegen (19) - HashAggregate [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cp_catalog_page_sk,cs_catalog_page_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit] + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_promo_sk] + Project [cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] InputAdapter - SortMergeJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] WholeStageCodegen (12) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #12 WholeStageCodegen (11) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_sold_date_sk] + Filter [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_order_number,cs_promo_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit] WholeStageCodegen (14) Sort [cr_item_sk,cr_order_number] InputAdapter @@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cr_item_sk,cr_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_net_loss,cr_order_number,cr_return_amount] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [i_item_sk] #7 InputAdapter @@ -136,33 +136,33 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] WholeStageCodegen (30) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,web_site_id] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [web_site_id] #15 WholeStageCodegen (29) - HashAggregate [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_web_site_sk] - BroadcastHashJoin [p_promo_sk,ws_promo_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_promo_sk,ws_web_site_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] InputAdapter - SortMergeJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] WholeStageCodegen (22) Sort [ws_item_sk,ws_order_number] InputAdapter Exchange [ws_item_sk,ws_order_number] #16 WholeStageCodegen (21) - Filter [ws_item_sk,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + Filter [ws_sold_date_sk,ws_web_site_sk,ws_item_sk,ws_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_order_number,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit] WholeStageCodegen (24) Sort [wr_item_sk,wr_order_number] InputAdapter @@ -171,7 +171,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [wr_item_sk,wr_order_number] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [i_item_sk] #7 InputAdapter @@ -184,22 +184,22 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] WholeStageCodegen (65) - HashAggregate [channel,isEmpty,isEmpty,isEmpty,sum,sum,sum] [id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #19 WholeStageCodegen (64) - HashAggregate [channel,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 WholeStageCodegen (100) - HashAggregate [isEmpty,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #20 WholeStageCodegen (99) - HashAggregate [profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index e6f723af7fd0e..602a670a49116 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -1,55 +1,55 @@ -TakeOrderedAndProject [channel,id,profit,returns,sales] +TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (75) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #1 + Exchange [channel,id,sales,returns,profit] #1 WholeStageCodegen (74) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (49) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter - Exchange [channel,id,profit,returns,sales] #2 + Exchange [channel,id,sales,returns,profit] #2 WholeStageCodegen (48) - HashAggregate [channel,id,profit,returns,sales] + HashAggregate [channel,id,sales,returns,profit] InputAdapter Union WholeStageCodegen (23) - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #3 WholeStageCodegen (22) - HashAggregate [channel,id,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union WholeStageCodegen (7) - HashAggregate [isEmpty,isEmpty,s_store_id,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [s_store_id] #4 WholeStageCodegen (6) - HashAggregate [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [p_promo_sk,ss_promo_sk] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_net_profit,ss_promo_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - Project [s_store_id,sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk] - BroadcastHashJoin [s_store_sk,ss_store_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_store_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [sr_net_loss,sr_return_amt,ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk] - BroadcastHashJoin [sr_item_sk,sr_ticket_number,ss_item_sk,ss_ticket_number] - Filter [ss_item_sk,ss_promo_sk,ss_sold_date_sk,ss_store_sk] + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + Filter [ss_sold_date_sk,ss_store_sk,ss_item_sk,ss_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_net_profit,ss_promo_sk,ss_sold_date_sk,ss_store_sk,ss_ticket_number] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Filter [sr_item_sk,sr_ticket_number] ColumnarToRow InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_net_loss,sr_return_amt,sr_ticket_number] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] InputAdapter BroadcastExchange #6 WholeStageCodegen (2) @@ -57,14 +57,14 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #7 WholeStageCodegen (3) Filter [s_store_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_store_id,s_store_sk] + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (4) @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [i_current_price,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_item_sk] + Scan parquet default.item [i_item_sk,i_current_price] InputAdapter BroadcastExchange #9 WholeStageCodegen (5) @@ -80,34 +80,34 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [p_channel_tv,p_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.promotion [p_channel_tv,p_promo_sk] + Scan parquet default.promotion [p_promo_sk,p_channel_tv] WholeStageCodegen (14) - HashAggregate [cp_catalog_page_id,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00))] + 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(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [cp_catalog_page_id] #10 WholeStageCodegen (13) - HashAggregate [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit] + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_net_profit,cs_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cp_catalog_page_id,cr_net_loss,cr_return_amount,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk] - BroadcastHashJoin [cp_catalog_page_sk,cs_catalog_page_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cr_net_loss,cr_return_amount,cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_promo_sk,cs_sold_date_sk] - BroadcastHashJoin [cr_item_sk,cr_order_number,cs_item_sk,cs_order_number] - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_sold_date_sk] + Project [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + Filter [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_item_sk,cs_net_profit,cs_order_number,cs_promo_sk,cs_sold_date_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit] InputAdapter BroadcastExchange #11 WholeStageCodegen (8) Filter [cr_item_sk,cr_order_number] ColumnarToRow InputAdapter - Scan parquet default.catalog_returns [cr_item_sk,cr_net_loss,cr_order_number,cr_return_amount] + Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -116,38 +116,38 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [cp_catalog_page_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_id,cp_catalog_page_sk] + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter ReusedExchange [i_item_sk] #8 InputAdapter ReusedExchange [p_promo_sk] #9 WholeStageCodegen (21) - HashAggregate [isEmpty,isEmpty,sum,sum,sum,web_site_id] [channel,id,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00))] + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true)),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [web_site_id] #13 WholeStageCodegen (20) - HashAggregate [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] [isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [p_promo_sk,ws_promo_sk] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_net_profit,ws_promo_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk] - Project [web_site_id,wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk] - BroadcastHashJoin [web_site_sk,ws_web_site_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk,ws_web_site_sk] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Project [wr_net_loss,wr_return_amt,ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - Filter [ws_item_sk,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + Filter [ws_sold_date_sk,ws_web_site_sk,ws_item_sk,ws_promo_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_ext_sales_price,ws_item_sk,ws_net_profit,ws_order_number,ws_promo_sk,ws_sold_date_sk,ws_web_site_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit] InputAdapter BroadcastExchange #14 WholeStageCodegen (15) Filter [wr_item_sk,wr_order_number] ColumnarToRow InputAdapter - Scan parquet default.web_returns [wr_item_sk,wr_net_loss,wr_order_number,wr_return_amt] + Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter @@ -156,26 +156,26 @@ TakeOrderedAndProject [channel,id,profit,returns,sales] Filter [web_site_sk] ColumnarToRow InputAdapter - Scan parquet default.web_site [web_site_id,web_site_sk] + Scan parquet default.web_site [web_site_sk,web_site_id] InputAdapter ReusedExchange [i_item_sk] #8 InputAdapter ReusedExchange [p_promo_sk] #9 WholeStageCodegen (47) - HashAggregate [channel,isEmpty,isEmpty,isEmpty,sum,sum,sum] [id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel] #16 WholeStageCodegen (46) - HashAggregate [channel,profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 WholeStageCodegen (73) - HashAggregate [isEmpty,isEmpty,isEmpty,sum,sum,sum] [channel,id,isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange #17 WholeStageCodegen (72) - HashAggregate [profit,returns,sales] [isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,isEmpty,sum,sum,sum,sum,sum,sum] - HashAggregate [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] [isEmpty,isEmpty,isEmpty,profit,returns,sales,sum,sum,sum,sum(profit),sum(returns),sum(sales)] + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - ReusedExchange [channel,id,isEmpty,isEmpty,isEmpty,sum,sum,sum] #3 + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/simplified.txt index 5ecbf84847937..2bd128100f527 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_sum] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] WholeStageCodegen (20) - Project [i_category,i_class,lochierarchy,rank_within_parent,total_sum] + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w0,lochierarchy,total_sum] + Window [total_sum,lochierarchy,_w0] WholeStageCodegen (19) - Sort [_w0,lochierarchy,total_sum] + Sort [lochierarchy,_w0,total_sum] InputAdapter - Exchange [_w0,lochierarchy] #1 + Exchange [lochierarchy,_w0] #1 WholeStageCodegen (18) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] [_w0] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] InputAdapter - Exchange [g_category,g_class,i_category,i_class,lochierarchy,total_sum] #2 + Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 WholeStageCodegen (17) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union WholeStageCodegen (11) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter - Exchange [g_category,g_class,i_category,i_class,lochierarchy,total_sum] #3 + Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #3 WholeStageCodegen (10) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [g_category,g_class,lochierarchy,sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] InputAdapter Exchange [i_category,i_class] #4 WholeStageCodegen (3) HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [i_category,i_class,ws_net_paid] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_net_paid] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -51,22 +51,22 @@ TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_ Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] WholeStageCodegen (9) - HashAggregate [i_category,isEmpty,sum] [g_category,g_class,i_class,isEmpty,lochierarchy,sum,sum(total_sum),total_sum] + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter Exchange [i_category] #7 WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [i_category,i_class,sum] [sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum] #4 WholeStageCodegen (16) - HashAggregate [isEmpty,sum] [g_category,g_class,i_category,i_class,isEmpty,lochierarchy,sum,sum(total_sum),total_sum] + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter Exchange #8 WholeStageCodegen (15) - HashAggregate [total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [i_category,i_class,sum] [sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 5ecbf84847937..2bd128100f527 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,47 +1,47 @@ -TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_sum] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] WholeStageCodegen (20) - Project [i_category,i_class,lochierarchy,rank_within_parent,total_sum] + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - Window [_w0,lochierarchy,total_sum] + Window [total_sum,lochierarchy,_w0] WholeStageCodegen (19) - Sort [_w0,lochierarchy,total_sum] + Sort [lochierarchy,_w0,total_sum] InputAdapter - Exchange [_w0,lochierarchy] #1 + Exchange [lochierarchy,_w0] #1 WholeStageCodegen (18) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] [_w0] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] InputAdapter - Exchange [g_category,g_class,i_category,i_class,lochierarchy,total_sum] #2 + Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 WholeStageCodegen (17) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union WholeStageCodegen (11) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter - Exchange [g_category,g_class,i_category,i_class,lochierarchy,total_sum] #3 + Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #3 WholeStageCodegen (10) - HashAggregate [g_category,g_class,i_category,i_class,lochierarchy,total_sum] + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [g_category,g_class,lochierarchy,sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] InputAdapter Exchange [i_category,i_class] #4 WholeStageCodegen (3) HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [i_category,i_class,ws_net_paid] - BroadcastHashJoin [i_item_sk,ws_item_sk] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [d_date_sk,ws_sold_date_sk] - Filter [ws_item_sk,ws_sold_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_sold_date_sk,ws_item_sk] ColumnarToRow InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_net_paid] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) Project [d_date_sk] - Filter [d_date_sk,d_month_seq] + Filter [d_month_seq,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_month_seq] @@ -51,22 +51,22 @@ TakeOrderedAndProject [i_category,i_class,lochierarchy,rank_within_parent,total_ Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_item_sk] + Scan parquet default.item [i_item_sk,i_class,i_category] WholeStageCodegen (9) - HashAggregate [i_category,isEmpty,sum] [g_category,g_class,i_class,isEmpty,lochierarchy,sum,sum(total_sum),total_sum] + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter Exchange [i_category] #7 WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [i_category,i_class,sum] [sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum] #4 WholeStageCodegen (16) - HashAggregate [isEmpty,sum] [g_category,g_class,i_category,i_class,isEmpty,lochierarchy,sum,sum(total_sum),total_sum] + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter Exchange #8 WholeStageCodegen (15) - HashAggregate [total_sum] [isEmpty,isEmpty,sum,sum] - HashAggregate [i_category,i_class,sum] [sum,sum(UnscaledValue(ws_net_paid)),total_sum] + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt index b2bd85daa5301..ded65becfe826 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt @@ -1,9 +1,9 @@ WholeStageCodegen (10) - Sort [i_category,i_class,i_item_desc,i_item_id,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - Exchange [i_category,i_class,i_item_desc,i_item_id,revenueratio] #1 + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 WholeStageCodegen (9) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w1,i_class] WholeStageCodegen (8) @@ -11,25 +11,25 @@ WholeStageCodegen (10) InputAdapter Exchange [i_class] #2 WholeStageCodegen (7) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #3 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (6) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] - SortMergeJoin [i_item_sk,ss_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (3) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 WholeStageCodegen (2) - Project [ss_ext_sales_price,ss_item_sk] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #5 WholeStageCodegen (1) @@ -37,7 +37,7 @@ WholeStageCodegen (10) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter WholeStageCodegen (5) Sort [i_item_sk] @@ -47,4 +47,4 @@ WholeStageCodegen (10) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 13113a1aec48a..0a4967373dcf5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,9 +1,9 @@ WholeStageCodegen (7) - Sort [i_category,i_class,i_item_desc,i_item_id,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter - Exchange [i_category,i_class,i_item_desc,i_item_id,revenueratio] #1 + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 WholeStageCodegen (6) - Project [_w0,_we0,i_category,i_class,i_current_price,i_item_desc,i_item_id,itemrevenue] + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w1,i_class] WholeStageCodegen (5) @@ -11,26 +11,26 @@ WholeStageCodegen (7) InputAdapter Exchange [i_class] #2 WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,sum] [_w0,_w1,itemrevenue,sum,sum(UnscaledValue(ss_ext_sales_price))] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,_w1,sum] InputAdapter - Exchange [i_category,i_class,i_current_price,i_item_desc,i_item_id] #3 + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 WholeStageCodegen (3) - HashAggregate [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] [sum,sum] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - Project [i_category,i_class,i_current_price,i_item_desc,i_item_id,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_ext_sales_price,ss_item_sk,ss_sold_date_sk] + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (1) Filter [i_category,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_category,i_class,i_current_price,i_item_desc,i_item_id,i_item_sk] + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -38,4 +38,4 @@ WholeStageCodegen (7) Filter [d_date,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date,d_date_sk] + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index 34fd1d24395cf..f78fc269986b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -199,7 +199,7 @@ trait PlanStabilitySuite extends TPCDSBase with DisableAdaptiveExecutionSuite { * "sum(sr_return_amt#14)", so we remove all of these using regex */ def cleanUpReferences(references: AttributeSet): String = { - referenceRegex.replaceAllIn(references.toSeq.map(_.name).sorted.mkString(","), "") + referenceRegex.replaceAllIn(references.map(_.name).mkString(","), "") } /**