diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 668a6618179e..9ea2f435d0eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -39,8 +39,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 -import org.apache.spark.sql.catalyst.trees.AlwaysProcess -import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.trees.{AlwaysProcess, CurrentOrigin} import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.util.{toPrettySQL, CharVarcharUtils} import org.apache.spark.sql.connector.catalog._ @@ -293,6 +292,7 @@ class Analyzer(override val catalogManager: CatalogManager) ResolveUnion :: typeCoercionRules ++ extendedResolutionRules : _*), + Batch("Remove TempResolvedColumn", Once, RemoveTempResolvedColumn), Batch("Apply Char Padding", Once, ApplyCharTypePadding), Batch("Post-Hoc Resolution", Once, @@ -623,28 +623,26 @@ class Analyzer(override val catalogManager: CatalogManager) // groupingExpressions for condition resolving. val aggForResolving = aggregate.copy(groupingExpressions = groupByExprs) // Try resolving the condition of the filter as though it is in the aggregate clause - val resolvedInfo = - ResolveAggregateFunctions.resolveFilterCondInAggregate(h.havingCondition, aggForResolving) + val (extraAggExprs, Seq(resolvedHavingCond)) = + ResolveAggregateFunctions.resolveExprsWithAggregate(Seq(h.havingCondition), aggForResolving) // Push the aggregate expressions into the aggregate (if any). - if (resolvedInfo.nonEmpty) { - val (extraAggExprs, resolvedHavingCond) = resolvedInfo.get - val newChild = constructAggregate(selectedGroupByExprs, groupByExprs, - aggregate.aggregateExpressions ++ extraAggExprs, aggregate.child) - - // Since the exprId of extraAggExprs will be changed in the constructed aggregate, and the - // aggregateExpressions keeps the input order. So here we build an exprMap to resolve the - // condition again. - val exprMap = extraAggExprs.zip( - newChild.asInstanceOf[Aggregate].aggregateExpressions.takeRight( - extraAggExprs.length)).toMap - val newCond = resolvedHavingCond.transform { - case ne: NamedExpression if exprMap.contains(ne) => exprMap(ne) - } + val newChild = constructAggregate(selectedGroupByExprs, groupByExprs, + aggregate.aggregateExpressions ++ extraAggExprs, aggregate.child) + + // Since the output exprId will be changed in the constructed aggregate, here we build an + // attrMap to resolve the condition again. + val attrMap = AttributeMap((aggForResolving.output ++ extraAggExprs.map(_.toAttribute)) + .zip(newChild.output)) + val newCond = resolvedHavingCond.transform { + case a: AttributeReference => attrMap.getOrElse(a, a) + } + + if (extraAggExprs.isEmpty) { + Filter(newCond, newChild) + } else { Project(newChild.output.dropRight(extraAggExprs.length), Filter(newCond, newChild)) - } else { - h } } @@ -2378,10 +2376,6 @@ class Analyzer(override val catalogManager: CatalogManager) */ def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning( _.containsPattern(PLAN_EXPRESSION), ruleId) { - // In case of HAVING (a filter after an aggregate) we use both the aggregate and - // its child for resolution. - case f @ Filter(_, a: Aggregate) if f.childrenResolved => - resolveSubQueries(f, Seq(a, a.child)) case j: LateralJoin if j.left.resolved => resolveSubQueries(j, j.children) // Only a few unary nodes (Project/Filter/Aggregate) can contain subqueries. @@ -2457,164 +2451,133 @@ class Analyzer(override val catalogManager: CatalogManager) _.containsPattern(AGGREGATE), ruleId) { // Resolve aggregate with having clause to Filter(..., Aggregate()). Note, to avoid wrongly // resolve the having condition expression, here we skip resolving it in ResolveReferences - // and transform it to Filter after aggregate is resolved. See more details in SPARK-31519. + // and transform it to Filter after aggregate is resolved. Basically columns in HAVING should + // be resolved with `agg.child.output` first. See more details in SPARK-31519. case UnresolvedHaving(cond, agg: Aggregate) if agg.resolved => - resolveHaving(Filter(cond, agg), agg) - - case f @ Filter(_, agg: Aggregate) if agg.resolved => - resolveHaving(f, agg) - - case sort @ Sort(sortOrder, global, aggregate: Aggregate) if aggregate.resolved => - - // Try resolving the ordering as though it is in the aggregate clause. - try { - // If a sort order is unresolved, containing references not in aggregate, or containing - // `AggregateExpression`, we need to push down it to the underlying aggregate operator. - val unresolvedSortOrders = sortOrder.filter { s => - !s.resolved || !s.references.subsetOf(aggregate.outputSet) || containsAggregate(s) - } - val aliasedOrdering = unresolvedSortOrders.map(o => Alias(o.child, "aggOrder")()) - - val aggregateWithExtraOrdering = aggregate.copy( - aggregateExpressions = aggregate.aggregateExpressions ++ aliasedOrdering) - - val resolvedAggregate: Aggregate = - executeSameContext(aggregateWithExtraOrdering).asInstanceOf[Aggregate] - - val (reResolvedAggExprs, resolvedAliasedOrdering) = - resolvedAggregate.aggregateExpressions.splitAt(aggregate.aggregateExpressions.length) - - // If we pass the analysis check, then the ordering expressions should only reference to - // aggregate expressions or grouping expressions, and it's safe to push them down to - // Aggregate. - checkAnalysis(resolvedAggregate) - - val originalAggExprs = aggregate.aggregateExpressions.map(trimNonTopLevelAliases) - - // If the ordering expression is same with original aggregate expression, we don't need - // to push down this ordering expression and can reference the original aggregate - // expression instead. - val needsPushDown = ArrayBuffer.empty[NamedExpression] - val orderToAlias = unresolvedSortOrders.zip(aliasedOrdering) - val evaluatedOrderings = - resolvedAliasedOrdering.asInstanceOf[Seq[Alias]].zip(orderToAlias).map { - case (evaluated, (order, aliasOrder)) => - val index = reResolvedAggExprs.indexWhere { - case Alias(child, _) => child semanticEquals evaluated.child - case other => other semanticEquals evaluated.child - } - - if (index == -1) { - if (hasCharVarchar(evaluated)) { - needsPushDown += aliasOrder - order.copy(child = aliasOrder) - } else { - needsPushDown += evaluated - order.copy(child = evaluated.toAttribute) - } - } else { - order.copy(child = originalAggExprs(index).toAttribute) - } + resolveOperatorWithAggregate(Seq(cond), agg, (newExprs, newChild) => { + Filter(newExprs.head, newChild) + }) + + case Filter(cond, agg: Aggregate) if agg.resolved => + // We should resolve the references normally based on child (agg.output) first. + val maybeResolved = resolveExpressionByPlanOutput(cond, agg) + resolveOperatorWithAggregate(Seq(maybeResolved), agg, (newExprs, newChild) => { + Filter(newExprs.head, newChild) + }) + + case Sort(sortOrder, global, agg: Aggregate) if agg.resolved => + // We should resolve the references normally based on child (agg.output) first. + val maybeResolved = sortOrder.map(_.child).map(resolveExpressionByPlanOutput(_, agg)) + resolveOperatorWithAggregate(maybeResolved, agg, (newExprs, newChild) => { + val newSortOrder = sortOrder.zip(newExprs).map { + case (sortOrder, expr) => sortOrder.copy(child = expr) } + Sort(newSortOrder, global, newChild) + }) + } - val sortOrdersMap = unresolvedSortOrders - .map(new TreeNodeRef(_)) - .zip(evaluatedOrderings) - .toMap - val finalSortOrders = sortOrder.map(s => sortOrdersMap.getOrElse(new TreeNodeRef(s), s)) + /** + * Resolves the given expressions as if they are in the given Aggregate operator, which means + * the column can be resolved using `agg.child` and aggregate functions/grouping columns are + * allowed. It returns a list of named expressions that need to be appended to + * `agg.aggregateExpressions`, and the list of resolved expressions. + */ + def resolveExprsWithAggregate( + exprs: Seq[Expression], + agg: Aggregate): (Seq[NamedExpression], Seq[Expression]) = { + def resolveCol(input: Expression): Expression = { + input.transform { + case u: UnresolvedAttribute => + try { + // Resolve the column and wrap it with `TempResolvedColumn`. If the resolved column + // doesn't end up with as aggregate function input or grouping column, we should + // undo the column resolution to avoid confusing error message. For example, if + // a table `t` has two columns `c1` and `c2`, for query `SELECT ... FROM t + // GROUP BY c1 HAVING c2 = 0`, even though we can resolve column `c2` here, we + // should undo it later and fail with "Column c2 not found". + agg.child.resolve(u.nameParts, resolver).map(TempResolvedColumn(_, u.nameParts)) + .getOrElse(u) + } catch { + case _: AnalysisException => u + } + } + } - // Since we don't rely on sort.resolved as the stop condition for this rule, - // we need to check this and prevent applying this rule multiple times - if (sortOrder == finalSortOrders) { - sort - } else { - Project(aggregate.output, - Sort(finalSortOrders, global, - aggregate.copy(aggregateExpressions = originalAggExprs ++ needsPushDown))) - } - } catch { - // Attempting to resolve in the aggregate can result in ambiguity. When this happens, - // just return the original plan. - case ae: AnalysisException => sort + def resolveSubQuery(input: Expression): Expression = { + if (SubqueryExpression.hasSubquery(input)) { + val fake = Project(Alias(input, "fake")() :: Nil, agg.child) + ResolveSubquery(fake).asInstanceOf[Project].projectList.head.asInstanceOf[Alias].child + } else { + input } - } + } - def hasCharVarchar(expr: Alias): Boolean = { - expr.find { - case ne: NamedExpression => CharVarcharUtils.getRawType(ne.metadata).nonEmpty - case _ => false - }.nonEmpty + val extraAggExprs = ArrayBuffer.empty[NamedExpression] + val transformed = exprs.map { e => + // Try resolving the expression as though it is in the aggregate clause. + val maybeResolved = resolveSubQuery(resolveCol(e)) + if (!maybeResolved.resolved) { + maybeResolved + } else { + buildAggExprList(maybeResolved, agg, extraAggExprs) + } + } + (extraAggExprs.toSeq, transformed) } - def containsAggregate(condition: Expression): Boolean = { - condition.find(_.isInstanceOf[AggregateExpression]).isDefined + private def trimTempResolvedField(input: Expression): Expression = input.transform { + case t: TempResolvedColumn => t.child } - def resolveFilterCondInAggregate( - filterCond: Expression, agg: Aggregate): Option[(Seq[NamedExpression], Expression)] = { - try { - val aggregatedCondition = - Aggregate( - agg.groupingExpressions, - Alias(filterCond, "havingCondition")() :: Nil, - agg.child) - val resolvedOperator = executeSameContext(aggregatedCondition) - def resolvedAggregateFilter = - resolvedOperator - .asInstanceOf[Aggregate] - .aggregateExpressions.head - - // If resolution was successful and we see the filter has an aggregate in it, add it to - // the original aggregate operator. - if (resolvedOperator.resolved) { - // Try to replace all aggregate expressions in the filter by an alias. - val aggregateExpressions = ArrayBuffer.empty[NamedExpression] - val transformedAggregateFilter = resolvedAggregateFilter.transform { - case ae: AggregateExpression => - val alias = Alias(ae, ae.toString)() - aggregateExpressions += alias - alias.toAttribute - // Grouping functions are handled in the rule [[ResolveGroupingAnalytics]]. - case e: Expression if agg.groupingExpressions.exists(_.semanticEquals(e)) && - !ResolveGroupingAnalytics.hasGroupingFunction(e) && - !agg.output.exists(_.semanticEquals(e)) => - e match { - case ne: NamedExpression => - aggregateExpressions += ne - ne.toAttribute - case _ => - val alias = Alias(e, e.toString)() - aggregateExpressions += alias - alias.toAttribute - } - } - if (aggregateExpressions.nonEmpty) { - Some(aggregateExpressions.toSeq, transformedAggregateFilter) - } else { - None - } - } else { - None + private def buildAggExprList( + expr: Expression, + agg: Aggregate, + aggExprList: ArrayBuffer[NamedExpression]): Expression = { + // Avoid adding an extra aggregate expression if it's already present in + // `agg.aggregateExpressions`. + val index = agg.aggregateExpressions.indexWhere { + case Alias(child, _) => child semanticEquals expr + case other => other semanticEquals expr + } + if (index >= 0) { + agg.aggregateExpressions(index).toAttribute + } else { + expr match { + case ae: AggregateExpression => + val cleaned = trimTempResolvedField(ae) + val alias = Alias(cleaned, cleaned.toString)() + aggExprList += alias + alias.toAttribute + case grouping: Expression if agg.groupingExpressions.exists(grouping.semanticEquals) => + trimTempResolvedField(grouping) match { + case ne: NamedExpression => + aggExprList += ne + ne.toAttribute + case other => + val alias = Alias(other, other.toString)() + aggExprList += alias + alias.toAttribute + } + case t: TempResolvedColumn => + // Undo the resolution as this column is neither inside aggregate functions nor a + // grouping column. It shouldn't be resolved with `agg.child.output`. + CurrentOrigin.withOrigin(t.origin)(UnresolvedAttribute(t.nameParts)) + case other => + other.withNewChildren(other.children.map(buildAggExprList(_, agg, aggExprList))) } - } catch { - // Attempting to resolve in the aggregate can result in ambiguity. When this happens, - // just return None and the caller side will return the original plan. - case ae: AnalysisException => None } } - def resolveHaving(filter: Filter, agg: Aggregate): LogicalPlan = { - // Try resolving the condition of the filter as though it is in the aggregate clause - val resolvedInfo = resolveFilterCondInAggregate(filter.condition, agg) - - // Push the aggregate expressions into the aggregate (if any). - if (resolvedInfo.nonEmpty) { - val (aggregateExpressions, resolvedHavingCond) = resolvedInfo.get - Project(agg.output, - Filter(resolvedHavingCond, - agg.copy(aggregateExpressions = agg.aggregateExpressions ++ aggregateExpressions))) + def resolveOperatorWithAggregate( + exprs: Seq[Expression], + agg: Aggregate, + buildOperator: (Seq[Expression], Aggregate) => LogicalPlan): LogicalPlan = { + val (extraAggExprs, resolvedExprs) = resolveExprsWithAggregate(exprs, agg) + if (extraAggExprs.isEmpty) { + buildOperator(resolvedExprs, agg) } else { - filter + Project(agg.output, buildOperator(resolvedExprs, agg.copy( + aggregateExpressions = agg.aggregateExpressions ++ extraAggExprs))) } } } @@ -4133,3 +4096,14 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { if (targetLength > charLength) StringRPad(expr, Literal(targetLength)) else expr } } + +/** + * Removes all [[TempResolvedColumn]]s in the query plan. This is the last resort, in case some + * rules in the main resolution batch miss to remove [[TempResolvedColumn]]s. We should run this + * rule right after the main resolution batch. + */ +object RemoveTempResolvedColumn extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions { + case t: TempResolvedColumn => UnresolvedAttribute(t.nameParts) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 1f7dd8bed3e0..a351a100e767 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -839,9 +839,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { def checkMixedReferencesInsideAggregateExpr(expr: Expression): Unit = { expr.foreach { case a: AggregateExpression if containsOuter(a) => - val outer = a.collect { case OuterReference(e) => e.toAttribute } - val local = a.references -- outer - if (local.nonEmpty) { + if (a.references.nonEmpty) { throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql) } case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 2fba15a3ca80..d84d735463ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -652,3 +652,15 @@ case object UnresolvedSeed extends LeafExpression with Unevaluable { override def dataType: DataType = throw new UnresolvedException("dataType") override lazy val resolved = false } + +/** + * An intermediate expression to hold a resolved (nested) column. Some rules may need to undo the + * column resolution and use this expression to keep the original column name. + */ +case class TempResolvedColumn(child: Expression, nameParts: Seq[String]) extends UnaryExpression + with Unevaluable { + override lazy val canonicalized = child.canonicalized + override def dataType: DataType = child.dataType + override protected def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index d1e953dc020a..ec68a6422d11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -192,9 +192,7 @@ object SubExprUtils extends PredicateHelper { val outerExpressions = ArrayBuffer.empty[Expression] def collectOutRefs(input: Expression): Unit = input match { case a: AggregateExpression if containsOuter(a) => - val outer = a.collect { case OuterReference(e) => e.toAttribute } - val local = a.references -- outer - if (local.nonEmpty) { + if (a.references.nonEmpty) { throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql) } else { // Collect and update the sub-tree so that outer references inside this aggregate diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 33eaea29ecb8..ab9c827d3282 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -325,8 +325,6 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper */ def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning( _.containsPattern(PLAN_EXPRESSION)) { - case f @ Filter(_, a: Aggregate) => - rewriteSubQueries(f, Seq(a, a.child)) case j: LateralJoin => val newPlan = rewriteSubQueries(j, j.children) // Since a lateral join's output depends on its left child output and its lateral subquery's diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index dc9f92d7c642..5cef24310e35 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -170,10 +170,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { val b = testRelation2.output(1) val c = testRelation2.output(2) val alias_a3 = count(a).as("a3") - val alias_b = b.as("aggOrder") // Case 1: when the child of Sort is not Aggregate, - // the sort reference is handled by the rule ResolveSortReferences + // the sort reference is handled by the rule ResolveMissingReferences val plan1 = testRelation2 .groupBy($"a", $"c", $"b")($"a", $"c", count($"a").as("a3")) .select($"a", $"c", $"a3") @@ -194,8 +193,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { .orderBy($"b".asc) val expected2 = testRelation2 - .groupBy(a, c, b)(a, c, alias_a3, alias_b) - .orderBy(alias_b.toAttribute.asc) + .groupBy(a, c, b)(a, c, alias_a3, b) + .orderBy(b.asc) .select(a, c, alias_a3.toAttribute) checkAnalysis(plan2, expected2) @@ -415,7 +414,6 @@ class AnalysisSuite extends AnalysisTest with Matchers { val expected = testRelation2 .groupBy(a, c)(alias1, alias2, alias3) .orderBy(alias1.toAttribute.asc, alias2.toAttribute.asc) - .select(alias1.toAttribute, alias2.toAttribute, alias3.toAttribute) checkAnalysis(plan, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala index ae36ab95b693..d2d7995848be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala @@ -287,9 +287,9 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { Seq(GroupingSets(Seq(Seq(), Seq(unresolved_a), Seq(unresolved_a, unresolved_b)))), Seq(unresolved_a, unresolved_b), r1)) val expected = Project(Seq(a, b), Sort( - Seq(SortOrder('aggOrder.byte.withNullability(false), Ascending)), true, + Seq(SortOrder(grouping_a, Ascending)), true, Aggregate(Seq(a, b, gid), - Seq(a, b, grouping_a.as("aggOrder")), + Seq(a, b, gid), Expand( Seq(Seq(a, b, c, nulInt, nulStr, 3L), Seq(a, b, c, a, nulStr, 1L), Seq(a, b, c, a, b, 0L)), @@ -309,9 +309,9 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { Seq(GroupingSets(Seq(Seq(), Seq(unresolved_a), Seq(unresolved_a, unresolved_b)))), Seq(unresolved_a, unresolved_b), r1)) val expected3 = Project(Seq(a, b), Sort( - Seq(SortOrder('aggOrder.long.withNullability(false), Ascending)), true, + Seq(SortOrder(gid, Ascending)), true, Aggregate(Seq(a, b, gid), - Seq(a, b, gid.as("aggOrder")), + Seq(a, b, gid), Expand( Seq(Seq(a, b, c, nulInt, nulStr, 3L), Seq(a, b, c, a, nulStr, 1L), Seq(a, b, c, a, b, 0L)), diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 9da7056c0ccc..f5e5b46d29ce 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -149,14 +149,13 @@ EXPLAIN FORMATTED struct -- !query output == Physical Plan == -AdaptiveSparkPlan (8) -+- Project (7) - +- Filter (6) - +- HashAggregate (5) - +- Exchange (4) - +- HashAggregate (3) - +- Filter (2) - +- Scan parquet default.explain_temp1 (1) +AdaptiveSparkPlan (7) ++- Filter (6) + +- HashAggregate (5) + +- Exchange (4) + +- HashAggregate (3) + +- Filter (2) + +- Scan parquet default.explain_temp1 (1) (1) Scan parquet default.explain_temp1 @@ -186,17 +185,13 @@ Input [2]: [key#x, max#x] Keys [1]: [key#x] Functions [1]: [max(val#x)] Aggregate Attributes [1]: [max(val#x)#x] -Results [3]: [key#x, max(val#x)#x AS max(val)#x, max(val#x)#x AS max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] (6) Filter -Input [3]: [key#x, max(val)#x, max(val#x)#x] -Condition : (isnotnull(max(val#x)#x) AND (max(val#x)#x > 0)) - -(7) Project -Output [2]: [key#x, max(val)#x] -Input [3]: [key#x, max(val)#x, max(val#x)#x] +Input [2]: [key#x, max(val)#x] +Condition : (isnotnull(max(val)#x) AND (max(val)#x > 0)) -(8) AdaptiveSparkPlan +(7) AdaptiveSparkPlan Output [2]: [key#x, max(val)#x] Arguments: isFinalPlan=false diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index ec7b15f01700..4e552d51a395 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -148,14 +148,13 @@ EXPLAIN FORMATTED struct -- !query output == Physical Plan == -* Project (8) -+- * Filter (7) - +- * HashAggregate (6) - +- Exchange (5) - +- * HashAggregate (4) - +- * Filter (3) - +- * ColumnarToRow (2) - +- Scan parquet default.explain_temp1 (1) +* Filter (7) ++- * HashAggregate (6) + +- Exchange (5) + +- * HashAggregate (4) + +- * Filter (3) + +- * ColumnarToRow (2) + +- Scan parquet default.explain_temp1 (1) (1) Scan parquet default.explain_temp1 @@ -188,15 +187,11 @@ Input [2]: [key#x, max#x] Keys [1]: [key#x] Functions [1]: [max(val#x)] Aggregate Attributes [1]: [max(val#x)#x] -Results [3]: [key#x, max(val#x)#x AS max(val)#x, max(val#x)#x AS max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] (7) Filter [codegen id : 2] -Input [3]: [key#x, max(val)#x, max(val#x)#x] -Condition : (isnotnull(max(val#x)#x) AND (max(val#x)#x > 0)) - -(8) Project [codegen id : 2] -Output [2]: [key#x, max(val)#x] -Input [3]: [key#x, max(val)#x, max(val#x)#x] +Input [2]: [key#x, max(val)#x] +Condition : (isnotnull(max(val)#x) AND (max(val)#x > 0)) -- !query 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 f2d08a531379..8dd51bf58b24 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 @@ -536,38 +536,38 @@ Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [5]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69] (89) Filter [codegen id : 46] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [5]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Condition : (isnotnull(sales#68) AND (cast(sales#68 as decimal(32,6)) > cast(Subquery scalar-subquery#70, [id=#71] as decimal(32,6)))) (90) Project [codegen id : 46] -Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [6]: [sales#68, number_sales#69, store AS channel#72, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [5]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Output [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#76), dynamicpruningexpression(cs_sold_date_sk#76 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 47] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] (93) Filter [codegen id : 47] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Condition : isnotnull(cs_item_sk#74) +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] +Condition : isnotnull(cs_item_sk#73) (94) Exchange -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] +Arguments: hashpartitioning(cs_item_sk#73, 5), ENSURE_REQUIREMENTS, [id=#77] (95) Sort [codegen id : 48] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] +Arguments: [cs_item_sk#73 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] @@ -577,82 +577,82 @@ Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#74] +Left keys [1]: [cs_item_sk#73] Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#78] (100) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [cs_sold_date_sk#76] +Right keys [1]: [d_date_sk#78] Join condition: None (101) Project [codegen id : 91] -Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] -Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] +Output [3]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75] +Input [5]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76, d_date_sk#78] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] (103) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [i_item_sk#80] +Left keys [1]: [cs_item_sk#73] +Right keys [1]: [i_item_sk#79] Join condition: None (104) Project [codegen id : 91] -Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [5]: [cs_quantity#74, cs_list_price#75, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [7]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] (105) HashAggregate [codegen id : 91] -Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Input [5]: [cs_quantity#74, cs_list_price#75, i_brand_id#80, i_class_id#81, i_category_id#82] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] (106) Exchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] (107) HashAggregate [codegen id : 92] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [5]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93] (108) Filter [codegen id : 92] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [5]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#70, [id=#71] as decimal(32,6)))) (109) Project [codegen id : 92] -Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [sales#92, number_sales#93, catalog AS channel#94, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [5]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Output [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#98), dynamicpruningexpression(ws_sold_date_sk#98 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (111) ColumnarToRow [codegen id : 93] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] (112) Filter [codegen id : 93] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Condition : isnotnull(ws_item_sk#97) +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +Condition : isnotnull(ws_item_sk#95) (113) Exchange -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +Arguments: hashpartitioning(ws_item_sk#95, 5), ENSURE_REQUIREMENTS, [id=#99] (114) Sort [codegen id : 94] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +Arguments: [ws_item_sk#95 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] @@ -662,91 +662,91 @@ Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#97] +Left keys [1]: [ws_item_sk#95] Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#100] (119) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None (120) Project [codegen id : 137] -Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] -Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] +Output [3]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97] +Input [5]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98, d_date_sk#100] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [4]: [i_item_sk#101, i_brand_id#102, i_class_id#103, i_category_id#104] (122) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ws_item_sk#95] +Right keys [1]: [i_item_sk#101] Join condition: None (123) Project [codegen id : 137] -Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [5]: [ws_quantity#96, ws_list_price#97, i_brand_id#102, i_class_id#103, i_category_id#104] +Input [7]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, i_item_sk#101, i_brand_id#102, i_class_id#103, i_category_id#104] (124) HashAggregate [codegen id : 137] -Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Input [5]: [ws_quantity#96, ws_list_price#97, i_brand_id#102, i_class_id#103, i_category_id#104] +Keys [3]: [i_brand_id#102, i_class_id#103, i_category_id#104] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#105, isEmpty#106, count#107] +Results [6]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum#108, isEmpty#109, count#110] (125) Exchange -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [6]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum#108, isEmpty#109, count#110] +Arguments: hashpartitioning(i_brand_id#102, i_class_id#103, i_category_id#104, 5), ENSURE_REQUIREMENTS, [id=#111] (126) HashAggregate [codegen id : 138] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Input [6]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum#108, isEmpty#109, count#110] +Keys [3]: [i_brand_id#102, i_class_id#103, i_category_id#104] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true))#112, count(1)#113] +Results [5]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true))#112 AS sales#114, count(1)#113 AS number_sales#115] (127) Filter [codegen id : 138] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [5]: [i_brand_id#102, i_class_id#103, i_category_id#104, sales#114, number_sales#115] +Condition : (isnotnull(sales#114) AND (cast(sales#114 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#70, [id=#71] as decimal(32,6)))) (128) Project [codegen id : 138] -Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Output [6]: [sales#114, number_sales#115, web AS channel#116, i_brand_id#102, i_class_id#103, i_category_id#104] +Input [5]: [i_brand_id#102, i_class_id#103, i_category_id#104, sales#114, number_sales#115] (129) Union (130) Expand [codegen id : 139] -Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1], [sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3], [sales#68, number_sales#69, channel#73, null, null, null, 7], [sales#68, number_sales#69, null, null, null, null, 15]], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Input [6]: [sales#68, number_sales#69, channel#72, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [[sales#68, number_sales#69, channel#72, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#68, number_sales#69, channel#72, i_brand_id#54, i_class_id#55, null, 1], [sales#68, number_sales#69, channel#72, i_brand_id#54, null, null, 3], [sales#68, number_sales#69, channel#72, null, null, null, 7], [sales#68, number_sales#69, null, null, null, null, 15]], [sales#68, number_sales#69, channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121] (131) HashAggregate [codegen id : 139] -Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Input [7]: [sales#68, number_sales#69, channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121] +Keys [5]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121] Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#125, isEmpty#126, sum#127] -Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Aggregate Attributes [3]: [sum#122, isEmpty#123, sum#124] +Results [8]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121, sum#125, isEmpty#126, sum#127] (132) Exchange -Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] -Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121, sum#125, isEmpty#126, sum#127] +Arguments: hashpartitioning(channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#128] (133) HashAggregate [codegen id : 140] -Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] -Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Input [8]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121, sum#125, isEmpty#126, sum#127] +Keys [5]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, spark_grouping_id#121] Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#132, sum(number_sales#69)#133] -Results [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales#68)#132 AS sum(sales)#134, sum(number_sales#69)#133 AS sum(number_sales)#135] +Aggregate Attributes [2]: [sum(sales#68)#129, sum(number_sales#69)#130] +Results [6]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, sum(sales#68)#129 AS sum(sales)#131, sum(number_sales#69)#130 AS sum(number_sales)#132] (134) TakeOrderedAndProject -Input [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] -Arguments: 100, [channel#120 ASC NULLS FIRST, i_brand_id#121 ASC NULLS FIRST, i_class_id#122 ASC NULLS FIRST, i_category_id#123 ASC NULLS FIRST], [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] +Input [6]: [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, sum(sales)#131, sum(number_sales)#132] +Arguments: 100, [channel#117 ASC NULLS FIRST, i_brand_id#118 ASC NULLS FIRST, i_class_id#119 ASC NULLS FIRST, i_category_id#120 ASC NULLS FIRST], [channel#117, i_brand_id#118, i_class_id#119, i_category_id#120, sum(sales)#131, sum(number_sales)#132] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#70, [id=#71] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -773,120 +773,120 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (135) Scan parquet default.store_sales -Output [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] +Output [3]: [ss_quantity#133, ss_list_price#134, ss_sold_date_sk#135] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#138), dynamicpruningexpression(ss_sold_date_sk#138 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ss_sold_date_sk#135), dynamicpruningexpression(ss_sold_date_sk#135 IN dynamicpruning#136)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] +Input [3]: [ss_quantity#133, ss_list_price#134, ss_sold_date_sk#135] (137) Scan parquet default.date_dim -Output [2]: [d_date_sk#140, d_year#141] +Output [2]: [d_date_sk#137, d_year#138] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#140, d_year#141] +Input [2]: [d_date_sk#137, d_year#138] (139) Filter [codegen id : 1] -Input [2]: [d_date_sk#140, d_year#141] -Condition : (((isnotnull(d_year#141) AND (d_year#141 >= 1999)) AND (d_year#141 <= 2001)) AND isnotnull(d_date_sk#140)) +Input [2]: [d_date_sk#137, d_year#138] +Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#137)) (140) Project [codegen id : 1] -Output [1]: [d_date_sk#140] -Input [2]: [d_date_sk#140, d_year#141] +Output [1]: [d_date_sk#137] +Input [2]: [d_date_sk#137, d_year#138] (141) BroadcastExchange -Input [1]: [d_date_sk#140] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#142] +Input [1]: [d_date_sk#137] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#139] (142) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#138] -Right keys [1]: [d_date_sk#140] +Left keys [1]: [ss_sold_date_sk#135] +Right keys [1]: [d_date_sk#137] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#136 AS quantity#143, ss_list_price#137 AS list_price#144] -Input [4]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138, d_date_sk#140] +Output [2]: [ss_quantity#133 AS quantity#140, ss_list_price#134 AS list_price#141] +Input [4]: [ss_quantity#133, ss_list_price#134, ss_sold_date_sk#135, d_date_sk#137] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] +Output [3]: [cs_quantity#142, cs_list_price#143, cs_sold_date_sk#144] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#147), dynamicpruningexpression(cs_sold_date_sk#147 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(cs_sold_date_sk#144), dynamicpruningexpression(cs_sold_date_sk#144 IN dynamicpruning#136)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] +Input [3]: [cs_quantity#142, cs_list_price#143, cs_sold_date_sk#144] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#148] +Output [1]: [d_date_sk#145] (147) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#147] -Right keys [1]: [d_date_sk#148] +Left keys [1]: [cs_sold_date_sk#144] +Right keys [1]: [d_date_sk#145] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#145 AS quantity#149, cs_list_price#146 AS list_price#150] -Input [4]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147, d_date_sk#148] +Output [2]: [cs_quantity#142 AS quantity#146, cs_list_price#143 AS list_price#147] +Input [4]: [cs_quantity#142, cs_list_price#143, cs_sold_date_sk#144, d_date_sk#145] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] +Output [3]: [ws_quantity#148, ws_list_price#149, ws_sold_date_sk#150] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ws_sold_date_sk#150), dynamicpruningexpression(ws_sold_date_sk#150 IN dynamicpruning#136)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] +Input [3]: [ws_quantity#148, ws_list_price#149, ws_sold_date_sk#150] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#154] +Output [1]: [d_date_sk#151] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#153] -Right keys [1]: [d_date_sk#154] +Left keys [1]: [ws_sold_date_sk#150] +Right keys [1]: [d_date_sk#151] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#151 AS quantity#155, ws_list_price#152 AS list_price#156] -Input [4]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153, d_date_sk#154] +Output [2]: [ws_quantity#148 AS quantity#152, ws_list_price#149 AS list_price#153] +Input [4]: [ws_quantity#148, ws_list_price#149, ws_sold_date_sk#150, d_date_sk#151] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#143, list_price#144] +Input [2]: [quantity#140, list_price#141] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#157, count#158] -Results [2]: [sum#159, count#160] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#140 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#141 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#154, count#155] +Results [2]: [sum#156, count#157] (156) Exchange -Input [2]: [sum#159, count#160] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] +Input [2]: [sum#156, count#157] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#158] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#159, count#160] +Input [2]: [sum#156, count#157] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162 AS average_sales#163] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#140 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#141 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#140 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#141 as decimal(12,2)))), DecimalType(18,2), true))#159] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#140 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#141 as decimal(12,2)))), DecimalType(18,2), true))#159 AS average_sales#160] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#138 IN dynamicpruning#139 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#135 IN dynamicpruning#136 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#140] +Output [1]: [d_date_sk#137] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#147 IN dynamicpruning#139 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#144 IN dynamicpruning#136 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#139 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#150 IN dynamicpruning#136 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) @@ -906,12 +906,12 @@ Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN d Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#70, [id=#71] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#76 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#70, [id=#71] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#98 IN dynamicpruning#5 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 bf3b205bb8c5..dd4f5c7f7e15 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Union WholeStageCodegen (46) Project [sales,number_sales,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))] + Filter [sales] Subquery #3 WholeStageCodegen (8) 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] @@ -54,7 +54,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #4 InputAdapter ReusedExchange [d_date_sk] #20 - 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),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] + 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),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (45) @@ -212,9 +212,9 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [ss_item_sk] #18 WholeStageCodegen (92) Project [sales,number_sales,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))] + Filter [sales] ReusedSubquery [average_sales] #3 - 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),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] + 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),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 WholeStageCodegen (91) @@ -246,9 +246,9 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (138) Project [sales,number_sales,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))] + Filter [sales] ReusedSubquery [average_sales] #3 - 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),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] + 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),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 WholeStageCodegen (137) 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 5e525515e382..02b000bf73fe 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 @@ -465,193 +465,193 @@ Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Results [5]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [5]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Condition : (isnotnull(sales#63) AND (cast(sales#63 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#63, number_sales#64, store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Output [6]: [sales#63, number_sales#64, store AS channel#67, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Output [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#71), dynamicpruningexpression(cs_sold_date_sk#71 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#69) +Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] +Condition : isnotnull(cs_item_sk#68) (81) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] +Left keys [1]: [cs_item_sk#68] Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [4]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [i_item_sk#73] +Left keys [1]: [cs_item_sk#68] +Right keys [1]: [i_item_sk#72] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [6]: [cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, i_brand_id#73, i_class_id#74, i_category_id#75] +Input [8]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#76] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#72] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#71] +Right keys [1]: [d_date_sk#76] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] +Output [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#73, i_class_id#74, i_category_id#75] +Input [7]: [cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, i_brand_id#73, i_class_id#74, i_category_id#75, d_date_sk#76] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Input [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#73, i_class_id#74, i_category_id#75] +Keys [3]: [i_brand_id#73, i_class_id#74, i_category_id#75] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum#80, isEmpty#81, count#82] (90) Exchange -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [6]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, 5), ENSURE_REQUIREMENTS, [id=#83] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Input [6]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#73, i_class_id#74, i_category_id#75] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] +Results [5]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [5]: [i_brand_id#73, i_class_id#74, i_category_id#75, sales#86, number_sales#87] +Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#87, number_sales#88, catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Output [6]: [sales#86, number_sales#87, catalog AS channel#88, i_brand_id#73, i_class_id#74, i_category_id#75] +Input [5]: [i_brand_id#73, i_class_id#74, i_category_id#75, sales#86, number_sales#87] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Output [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -Condition : isnotnull(ws_item_sk#91) +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_item_sk#89) (97) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] +Left keys [1]: [ws_item_sk#89] Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [4]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [i_item_sk#95] +Left keys [1]: [ws_item_sk#89] +Right keys [1]: [i_item_sk#93] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [6]: [ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, i_brand_id#94, i_class_id#95, i_category_id#96] +Input [8]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#99] +Output [1]: [d_date_sk#97] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#99] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#97] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] +Output [5]: [ws_quantity#90, ws_list_price#91, i_brand_id#94, i_class_id#95, i_category_id#96] +Input [7]: [ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, i_brand_id#94, i_class_id#95, i_category_id#96, d_date_sk#97] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Input [5]: [ws_quantity#90, ws_list_price#91, i_brand_id#94, i_class_id#95, i_category_id#96] +Keys [3]: [i_brand_id#94, i_class_id#95, i_category_id#96] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#98, isEmpty#99, count#100] +Results [6]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum#101, isEmpty#102, count#103] (106) Exchange -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [6]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum#101, isEmpty#102, count#103] +Arguments: hashpartitioning(i_brand_id#94, i_class_id#95, i_category_id#96, 5), ENSURE_REQUIREMENTS, [id=#104] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Input [6]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum#101, isEmpty#102, count#103] +Keys [3]: [i_brand_id#94, i_class_id#95, i_category_id#96] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true))#105, count(1)#106] +Results [5]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true))#105 AS sales#107, count(1)#106 AS number_sales#108] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [5]: [i_brand_id#94, i_class_id#95, i_category_id#96, sales#107, number_sales#108] +Condition : (isnotnull(sales#107) AND (cast(sales#107 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#109, number_sales#110, web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Output [6]: [sales#107, number_sales#108, web AS channel#109, i_brand_id#94, i_class_id#95, i_category_id#96] +Input [5]: [i_brand_id#94, i_class_id#95, i_category_id#96, sales#107, number_sales#108] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: [[sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 0], [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, null, 1], [sales#63, number_sales#64, channel#68, i_brand_id#46, null, null, 3], [sales#63, number_sales#64, channel#68, null, null, null, 7], [sales#63, number_sales#64, null, null, null, null, 15]], [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Input [6]: [sales#63, number_sales#64, channel#67, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: [[sales#63, number_sales#64, channel#67, i_brand_id#46, i_class_id#47, i_category_id#48, 0], [sales#63, number_sales#64, channel#67, i_brand_id#46, i_class_id#47, null, 1], [sales#63, number_sales#64, channel#67, i_brand_id#46, null, null, 3], [sales#63, number_sales#64, channel#67, null, null, null, 7], [sales#63, number_sales#64, null, null, null, null, 15]], [sales#63, number_sales#64, channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Input [7]: [sales#63, number_sales#64, channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114] +Keys [5]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114] Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] -Results [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] +Results [8]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114, sum#118, isEmpty#119, sum#120] (113) Exchange -Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] -Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [8]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114, sum#118, isEmpty#119, sum#120] +Arguments: hashpartitioning(channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114, 5), ENSURE_REQUIREMENTS, [id=#121] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] -Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Input [8]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114, sum#118, isEmpty#119, sum#120] +Keys [5]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, spark_grouping_id#114] Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#125, sum(number_sales#64)#126] -Results [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales#63)#125 AS sum(sales)#127, sum(number_sales#64)#126 AS sum(number_sales)#128] +Aggregate Attributes [2]: [sum(sales#63)#122, sum(number_sales#64)#123] +Results [6]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum(sales#63)#122 AS sum(sales)#124, sum(number_sales#64)#123 AS sum(number_sales)#125] (115) TakeOrderedAndProject -Input [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] -Arguments: 100, [channel#113 ASC NULLS FIRST, i_brand_id#114 ASC NULLS FIRST, i_class_id#115 ASC NULLS FIRST, i_category_id#116 ASC NULLS FIRST], [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] +Input [6]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum(sales)#124, sum(number_sales)#125] +Arguments: 100, [channel#110 ASC NULLS FIRST, i_brand_id#111 ASC NULLS FIRST, i_class_id#112 ASC NULLS FIRST, i_category_id#113 ASC NULLS FIRST], [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum(sales)#124, sum(number_sales)#125] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#65, [id=#66] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -678,120 +678,120 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (116) Scan parquet default.store_sales -Output [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] +Output [3]: [ss_quantity#126, ss_list_price#127, ss_sold_date_sk#128] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(ss_sold_date_sk#128), dynamicpruningexpression(ss_sold_date_sk#128 IN dynamicpruning#129)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] +Input [3]: [ss_quantity#126, ss_list_price#127, ss_sold_date_sk#128] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#133, d_year#134] +Input [2]: [d_date_sk#130, d_year#131] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#133, d_year#134] -Condition : (((isnotnull(d_year#134) AND (d_year#134 >= 1999)) AND (d_year#134 <= 2001)) AND isnotnull(d_date_sk#133)) +Input [2]: [d_date_sk#130, d_year#131] +Condition : (((isnotnull(d_year#131) AND (d_year#131 >= 1999)) AND (d_year#131 <= 2001)) AND isnotnull(d_date_sk#130)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#133] -Input [2]: [d_date_sk#133, d_year#134] +Output [1]: [d_date_sk#130] +Input [2]: [d_date_sk#130, d_year#131] (122) BroadcastExchange -Input [1]: [d_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] +Input [1]: [d_date_sk#130] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] (123) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#131] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [ss_sold_date_sk#128] +Right keys [1]: [d_date_sk#130] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#129 AS quantity#136, ss_list_price#130 AS list_price#137] -Input [4]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131, d_date_sk#133] +Output [2]: [ss_quantity#126 AS quantity#133, ss_list_price#127 AS list_price#134] +Input [4]: [ss_quantity#126, ss_list_price#127, ss_sold_date_sk#128, d_date_sk#130] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] +Output [3]: [cs_quantity#135, cs_list_price#136, cs_sold_date_sk#137] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#140), dynamicpruningexpression(cs_sold_date_sk#140 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(cs_sold_date_sk#137), dynamicpruningexpression(cs_sold_date_sk#137 IN dynamicpruning#129)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] +Input [3]: [cs_quantity#135, cs_list_price#136, cs_sold_date_sk#137] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#141] +Output [1]: [d_date_sk#138] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#140] -Right keys [1]: [d_date_sk#141] +Left keys [1]: [cs_sold_date_sk#137] +Right keys [1]: [d_date_sk#138] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#138 AS quantity#142, cs_list_price#139 AS list_price#143] -Input [4]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140, d_date_sk#141] +Output [2]: [cs_quantity#135 AS quantity#139, cs_list_price#136 AS list_price#140] +Input [4]: [cs_quantity#135, cs_list_price#136, cs_sold_date_sk#137, d_date_sk#138] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] +Output [3]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#146), dynamicpruningexpression(ws_sold_date_sk#146 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#129)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] +Input [3]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#147] +Output [1]: [d_date_sk#144] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#146] -Right keys [1]: [d_date_sk#147] +Left keys [1]: [ws_sold_date_sk#143] +Right keys [1]: [d_date_sk#144] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#144 AS quantity#148, ws_list_price#145 AS list_price#149] -Input [4]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146, d_date_sk#147] +Output [2]: [ws_quantity#141 AS quantity#145, ws_list_price#142 AS list_price#146] +Input [4]: [ws_quantity#141, ws_list_price#142, ws_sold_date_sk#143, d_date_sk#144] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#136, list_price#137] +Input [2]: [quantity#133, list_price#134] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#150, count#151] -Results [2]: [sum#152, count#153] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#133 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#134 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#147, count#148] +Results [2]: [sum#149, count#150] (137) Exchange -Input [2]: [sum#152, count#153] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#154] +Input [2]: [sum#149, count#150] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#151] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#152, count#153] +Input [2]: [sum#149, count#150] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155 AS average_sales#156] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#133 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#134 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#133 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#134 as decimal(12,2)))), DecimalType(18,2), true))#152] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#133 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#134 as decimal(12,2)))), DecimalType(18,2), true))#152 AS average_sales#153] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#128 IN dynamicpruning#129 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#133] +Output [1]: [d_date_sk#130] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#140 IN dynamicpruning#132 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#137 IN dynamicpruning#129 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#146 IN dynamicpruning#132 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#129 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) @@ -811,12 +811,12 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#71 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#5 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 9ae1a0e30e90..9de361b56dff 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Union WholeStageCodegen (26) Project [sales,number_sales,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))] + Filter [sales] Subquery #3 WholeStageCodegen (8) 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] @@ -54,7 +54,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #4 InputAdapter ReusedExchange [d_date_sk] #15 - 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),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] + 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),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (25) @@ -173,9 +173,9 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (52) Project [sales,number_sales,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))] + Filter [sales] ReusedSubquery [average_sales] #3 - 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),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] + 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),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 WholeStageCodegen (51) @@ -198,9 +198,9 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #3 WholeStageCodegen (78) Project [sales,number_sales,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))] + Filter [sales] ReusedSubquery [average_sales] #3 - 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),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] + 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),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #17 WholeStageCodegen (77) 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 d0bdd4104836..00cdd22094ad 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 @@ -1,120 +1,118 @@ == Physical Plan == -TakeOrderedAndProject (116) -+- * BroadcastHashJoin Inner BuildRight (115) - :- * Project (90) - : +- * Filter (89) - : +- * HashAggregate (88) - : +- Exchange (87) - : +- * HashAggregate (86) - : +- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * Project (74) - : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- * SortMergeJoin LeftSemi (67) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (66) - : : : +- Exchange (65) - : : : +- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (62) - : : : +- * HashAggregate (61) - : : : +- Exchange (60) - : : : +- * HashAggregate (59) - : : : +- * SortMergeJoin LeftSemi (58) - : : : :- * Sort (46) - : : : : +- Exchange (45) - : : : : +- * HashAggregate (44) - : : : : +- Exchange (43) - : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- * SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) - : : : +- * Sort (57) - : : : +- Exchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : :- * Filter (49) - : : : : : +- * ColumnarToRow (48) - : : : : : +- Scan parquet default.web_sales (47) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (72) - : : +- * Project (71) - : : +- * Filter (70) - : : +- * ColumnarToRow (69) - : : +- Scan parquet default.date_dim (68) - : +- BroadcastExchange (83) - : +- * SortMergeJoin LeftSemi (82) - : :- * Sort (79) - : : +- Exchange (78) - : : +- * Filter (77) - : : +- * ColumnarToRow (76) - : : +- Scan parquet default.item (75) - : +- * Sort (81) - : +- ReusedExchange (80) - +- BroadcastExchange (114) - +- * Project (113) - +- * Filter (112) - +- * HashAggregate (111) - +- Exchange (110) - +- * HashAggregate (109) - +- * Project (108) - +- * BroadcastHashJoin Inner BuildRight (107) - :- * Project (105) - : +- * BroadcastHashJoin Inner BuildRight (104) - : :- * SortMergeJoin LeftSemi (98) - : : :- * Sort (95) - : : : +- Exchange (94) - : : : +- * Filter (93) - : : : +- * ColumnarToRow (92) - : : : +- Scan parquet default.store_sales (91) - : : +- * Sort (97) - : : +- ReusedExchange (96) - : +- BroadcastExchange (103) - : +- * Project (102) - : +- * Filter (101) - : +- * ColumnarToRow (100) - : +- Scan parquet default.date_dim (99) - +- ReusedExchange (106) +TakeOrderedAndProject (114) ++- * BroadcastHashJoin Inner BuildRight (113) + :- * Filter (89) + : +- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- * SortMergeJoin LeftSemi (67) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- * Sort (66) + : : : +- Exchange (65) + : : : +- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.item (6) + : : : +- BroadcastExchange (62) + : : : +- * HashAggregate (61) + : : : +- Exchange (60) + : : : +- * HashAggregate (59) + : : : +- * SortMergeJoin LeftSemi (58) + : : : :- * Sort (46) + : : : : +- Exchange (45) + : : : : +- * HashAggregate (44) + : : : : +- Exchange (43) + : : : : +- * HashAggregate (42) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- * SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) + : : : +- * Sort (57) + : : : +- Exchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : :- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.web_sales (47) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (72) + : : +- * Project (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.date_dim (68) + : +- BroadcastExchange (83) + : +- * SortMergeJoin LeftSemi (82) + : :- * Sort (79) + : : +- Exchange (78) + : : +- * Filter (77) + : : +- * ColumnarToRow (76) + : : +- Scan parquet default.item (75) + : +- * Sort (81) + : +- ReusedExchange (80) + +- BroadcastExchange (112) + +- * Filter (111) + +- * HashAggregate (110) + +- Exchange (109) + +- * HashAggregate (108) + +- * Project (107) + +- * BroadcastHashJoin Inner BuildRight (106) + :- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * SortMergeJoin LeftSemi (97) + : : :- * Sort (94) + : : : +- Exchange (93) + : : : +- * Filter (92) + : : : +- * ColumnarToRow (91) + : : : +- Scan parquet default.store_sales (90) + : : +- * Sort (96) + : : +- ReusedExchange (95) + : +- BroadcastExchange (102) + : +- * Project (101) + : +- * Filter (100) + : +- * ColumnarToRow (99) + : +- Scan parquet default.date_dim (98) + +- ReusedExchange (105) (1) Scan parquet default.store_sales @@ -518,289 +516,281 @@ Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Results [6]: [store AS channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#70, count(1)#68 AS number_sales#71] (89) Filter [codegen id : 92] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +Input [6]: [channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sales#70, number_sales#71] +Condition : (isnotnull(sales#70) AND (cast(sales#70 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 92] -Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] - -(91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +(90) Scan parquet default.store_sales +Output [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +(91) ColumnarToRow [codegen id : 46] +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(93) Filter [codegen id : 46] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_item_sk#75) +(92) Filter [codegen id : 46] +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Condition : isnotnull(ss_item_sk#74) -(94) Exchange -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] +(93) Exchange +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Arguments: hashpartitioning(ss_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#79] -(95) Sort [codegen id : 47] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 +(94) Sort [codegen id : 47] +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Arguments: [ss_item_sk#74 ASC NULLS FIRST], false, 0 -(96) ReusedExchange [Reuses operator id: unknown] +(95) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 66] +(96) Sort [codegen id : 66] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#75] +(97) SortMergeJoin [codegen id : 90] +Left keys [1]: [ss_item_sk#74] Right keys [1]: [ss_item_sk#47] Join condition: None -(99) Scan parquet default.date_dim -Output [2]: [d_date_sk#81, d_week_seq#82] +(98) Scan parquet default.date_dim +Output [2]: [d_date_sk#80, d_week_seq#81] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 67] -Input [2]: [d_date_sk#81, d_week_seq#82] +(99) ColumnarToRow [codegen id : 67] +Input [2]: [d_date_sk#80, d_week_seq#81] -(101) Filter [codegen id : 67] -Input [2]: [d_date_sk#81, d_week_seq#82] -Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) +(100) Filter [codegen id : 67] +Input [2]: [d_date_sk#80, d_week_seq#81] +Condition : ((isnotnull(d_week_seq#81) AND (d_week_seq#81 = Subquery scalar-subquery#82, [id=#83])) AND isnotnull(d_date_sk#80)) -(102) Project [codegen id : 67] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_week_seq#82] +(101) Project [codegen id : 67] +Output [1]: [d_date_sk#80] +Input [2]: [d_date_sk#80, d_week_seq#81] -(103) BroadcastExchange -Input [1]: [d_date_sk#81] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] +(102) BroadcastExchange +Input [1]: [d_date_sk#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] -(104) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#81] +(103) BroadcastHashJoin [codegen id : 90] +Left keys [1]: [ss_sold_date_sk#77] +Right keys [1]: [d_date_sk#80] Join condition: None -(105) Project [codegen id : 90] -Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] -Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] +(104) Project [codegen id : 90] +Output [3]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76] +Input [5]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#80] -(106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +(105) ReusedExchange [Reuses operator id: 83] +Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(107) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [i_item_sk#86] +(106) BroadcastHashJoin [codegen id : 90] +Left keys [1]: [ss_item_sk#74] +Right keys [1]: [i_item_sk#85] Join condition: None -(108) Project [codegen id : 90] -Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] - -(109) HashAggregate [codegen id : 90] -Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] - -(110) Exchange -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] - -(111) HashAggregate [codegen id : 91] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] - -(112) Filter [codegen id : 91] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) - -(113) Project [codegen id : 91] -Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] - -(114) BroadcastExchange -Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] - -(115) BroadcastHashJoin [codegen id : 92] +(107) Project [codegen id : 90] +Output [5]: [ss_quantity#75, ss_list_price#76, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [7]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] + +(108) HashAggregate [codegen id : 90] +Input [5]: [ss_quantity#75, ss_list_price#76, i_brand_id#86, i_class_id#87, i_category_id#88] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#92, isEmpty#93, count#94] + +(109) Exchange +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] + +(110) HashAggregate [codegen id : 91] +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [store AS channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#99, count(1)#97 AS number_sales#100] + +(111) Filter [codegen id : 91] +Input [6]: [channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Condition : (isnotnull(sales#99) AND (cast(sales#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) + +(112) BroadcastExchange +Input [6]: [channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#101] + +(113) BroadcastHashJoin [codegen id : 92] Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] -Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Right keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] Join condition: None -(116) TakeOrderedAndProject -Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +(114) TakeOrderedAndProject +Input [12]: [channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sales#70, number_sales#71, channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sales#70, number_sales#71, channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] -* HashAggregate (139) -+- Exchange (138) - +- * HashAggregate (137) - +- Union (136) - :- * Project (125) - : +- * BroadcastHashJoin Inner BuildRight (124) - : :- * ColumnarToRow (118) - : : +- Scan parquet default.store_sales (117) - : +- BroadcastExchange (123) - : +- * Project (122) - : +- * Filter (121) - : +- * ColumnarToRow (120) - : +- Scan parquet default.date_dim (119) - :- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * ColumnarToRow (127) - : : +- Scan parquet default.catalog_sales (126) - : +- ReusedExchange (128) - +- * Project (135) - +- * BroadcastHashJoin Inner BuildRight (134) - :- * ColumnarToRow (132) - : +- Scan parquet default.web_sales (131) - +- ReusedExchange (133) - - -(117) Scan parquet default.store_sales -Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +* HashAggregate (137) ++- Exchange (136) + +- * HashAggregate (135) + +- Union (134) + :- * Project (123) + : +- * BroadcastHashJoin Inner BuildRight (122) + : :- * ColumnarToRow (116) + : : +- Scan parquet default.store_sales (115) + : +- BroadcastExchange (121) + : +- * Project (120) + : +- * Filter (119) + : +- * ColumnarToRow (118) + : +- Scan parquet default.date_dim (117) + :- * Project (128) + : +- * BroadcastHashJoin Inner BuildRight (127) + : :- * ColumnarToRow (125) + : : +- Scan parquet default.catalog_sales (124) + : +- ReusedExchange (126) + +- * Project (133) + +- * BroadcastHashJoin Inner BuildRight (132) + :- * ColumnarToRow (130) + : +- Scan parquet default.web_sales (129) + +- ReusedExchange (131) + + +(115) Scan parquet default.store_sales +Output [3]: [ss_quantity#102, ss_list_price#103, ss_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ss_sold_date_sk#104), dynamicpruningexpression(ss_sold_date_sk#104 IN dynamicpruning#105)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +(116) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#102, ss_list_price#103, ss_sold_date_sk#104] -(119) Scan parquet default.date_dim -Output [2]: [d_date_sk#108, d_year#109] +(117) Scan parquet default.date_dim +Output [2]: [d_date_sk#106, d_year#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] +(118) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#106, d_year#107] -(121) Filter [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] -Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1999)) AND (d_year#109 <= 2001)) AND isnotnull(d_date_sk#108)) +(119) Filter [codegen id : 1] +Input [2]: [d_date_sk#106, d_year#107] +Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1999)) AND (d_year#107 <= 2001)) AND isnotnull(d_date_sk#106)) -(122) Project [codegen id : 1] -Output [1]: [d_date_sk#108] -Input [2]: [d_date_sk#108, d_year#109] +(120) Project [codegen id : 1] +Output [1]: [d_date_sk#106] +Input [2]: [d_date_sk#106, d_year#107] -(123) BroadcastExchange -Input [1]: [d_date_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] +(121) BroadcastExchange +Input [1]: [d_date_sk#106] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] -(124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#106] -Right keys [1]: [d_date_sk#108] +(122) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#104] +Right keys [1]: [d_date_sk#106] Join condition: None -(125) Project [codegen id : 2] -Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] -Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] +(123) Project [codegen id : 2] +Output [2]: [ss_quantity#102 AS quantity#109, ss_list_price#103 AS list_price#110] +Input [4]: [ss_quantity#102, ss_list_price#103, ss_sold_date_sk#104, d_date_sk#106] -(126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +(124) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#111, cs_list_price#112, cs_sold_date_sk#113] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(cs_sold_date_sk#113), dynamicpruningexpression(cs_sold_date_sk#113 IN dynamicpruning#105)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +(125) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#111, cs_list_price#112, cs_sold_date_sk#113] -(128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#116] +(126) ReusedExchange [Reuses operator id: 121] +Output [1]: [d_date_sk#114] -(129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +(127) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#113] +Right keys [1]: [d_date_sk#114] Join condition: None -(130) Project [codegen id : 4] -Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] -Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] +(128) Project [codegen id : 4] +Output [2]: [cs_quantity#111 AS quantity#115, cs_list_price#112 AS list_price#116] +Input [4]: [cs_quantity#111, cs_list_price#112, cs_sold_date_sk#113, d_date_sk#114] -(131) Scan parquet default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +(129) Scan parquet default.web_sales +Output [3]: [ws_quantity#117, ws_list_price#118, ws_sold_date_sk#119] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#119), dynamicpruningexpression(ws_sold_date_sk#119 IN dynamicpruning#105)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +(130) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#117, ws_list_price#118, ws_sold_date_sk#119] -(133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#122] +(131) ReusedExchange [Reuses operator id: 121] +Output [1]: [d_date_sk#120] -(134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +(132) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#119] +Right keys [1]: [d_date_sk#120] Join condition: None -(135) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] +(133) Project [codegen id : 6] +Output [2]: [ws_quantity#117 AS quantity#121, ws_list_price#118 AS list_price#122] +Input [4]: [ws_quantity#117, ws_list_price#118, ws_sold_date_sk#119, d_date_sk#120] -(136) Union +(134) Union -(137) HashAggregate [codegen id : 7] -Input [2]: [quantity#111, list_price#112] +(135) HashAggregate [codegen id : 7] +Input [2]: [quantity#109, list_price#110] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#125, count#126] -Results [2]: [sum#127, count#128] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#123, count#124] +Results [2]: [sum#125, count#126] -(138) Exchange -Input [2]: [sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +(136) Exchange +Input [2]: [sum#125, count#126] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#127] -(139) HashAggregate [codegen id : 8] -Input [2]: [sum#127, count#128] +(137) HashAggregate [codegen id : 8] +Input [2]: [sum#125, count#126] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))#128] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))#128 AS average_sales#129] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 -ReusedExchange (140) +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 +ReusedExchange (138) -(140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +(138) ReusedExchange [Reuses operator id: 121] +Output [1]: [d_date_sk#106] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 +Subquery:3 Hosting operator id = 124 Hosting Expression = cs_sold_date_sk#113 IN dynamicpruning#105 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 +Subquery:4 Hosting operator id = 129 Hosting Expression = ws_sold_date_sk#119 IN dynamicpruning#105 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (141) +ReusedExchange (139) -(141) ReusedExchange [Reuses operator id: 72] +(139) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#49] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -ReusedExchange (142) +ReusedExchange (140) -(142) ReusedExchange [Reuses operator id: 16] +(140) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#14] Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 @@ -808,62 +798,62 @@ Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN d Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (146) -+- * Filter (145) - +- * ColumnarToRow (144) - +- Scan parquet default.date_dim (143) +* Project (144) ++- * Filter (143) + +- * ColumnarToRow (142) + +- Scan parquet default.date_dim (141) -(143) Scan parquet default.date_dim -Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +(141) Scan parquet default.date_dim +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +(142) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -(145) Filter [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 2000)) AND (d_moy#134 = 12)) AND (d_dom#135 = 11)) +(143) Filter [codegen id : 1] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 2000)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) -(146) Project [codegen id : 1] -Output [1]: [d_week_seq#132] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +(144) Project [codegen id : 1] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:10 Hosting operator id = 111 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 -ReusedExchange (147) +Subquery:11 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#78 +ReusedExchange (145) -(147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#81] +(145) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#80] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] -* Project (151) -+- * Filter (150) - +- * ColumnarToRow (149) - +- Scan parquet default.date_dim (148) +Subquery:12 Hosting operator id = 100 Hosting Expression = Subquery scalar-subquery#82, [id=#83] +* Project (149) ++- * Filter (148) + +- * ColumnarToRow (147) + +- Scan parquet default.date_dim (146) -(148) Scan parquet default.date_dim -Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +(146) Scan parquet default.date_dim +Output [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +(147) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] -(150) Filter [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] -Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1999)) AND (d_moy#138 = 12)) AND (d_dom#139 = 11)) +(148) Filter [codegen id : 1] +Input [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] +Condition : (((((isnotnull(d_year#135) AND isnotnull(d_moy#136)) AND isnotnull(d_dom#137)) AND (d_year#135 = 1999)) AND (d_moy#136 = 12)) AND (d_dom#137 = 11)) -(151) Project [codegen id : 1] -Output [1]: [d_week_seq#136] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +(149) Project [codegen id : 1] +Output [1]: [d_week_seq#134] +Input [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] 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 6a8fcada7a61..f1d25b6e5130 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,261 +1,259 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (92) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [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 #4 - WholeStageCodegen (8) - 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 #18 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - ReusedExchange [d_date_sk] #19 + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + 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 #18 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + ReusedExchange [d_date_sk] #19 + InputAdapter + BroadcastExchange #19 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #19 + Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #19 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #19 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (45) + 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] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (1) + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + InputAdapter + WholeStageCodegen (21) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (20) + Project [i_item_sk] + 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_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (19) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (22) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #3 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #19 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #19 - 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),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_class_id,i_category_id] #1 - WholeStageCodegen (45) - 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] - SortMergeJoin [ss_item_sk,ss_item_sk] + 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 #15 + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] + WholeStageCodegen (24) + Sort [i_item_sk] InputAdapter - Exchange [ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk] + Exchange [i_item_sk] #16 + WholeStageCodegen (23) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (21) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (20) - Project [i_item_sk] - 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] + ReusedExchange [ss_item_sk] #17 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (91) + Filter [sales] + ReusedSubquery [average_sales] #4 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #21 + WholeStageCodegen (90) + 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] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (47) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #22 + WholeStageCodegen (46) + Filter [ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (19) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - 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] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (22) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ColumnarToRow + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #6 + ReusedExchange [d_date_sk] #23 + InputAdapter + WholeStageCodegen (66) + Sort [ss_item_sk] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (44) - SortMergeJoin [i_item_sk,ss_item_sk] - InputAdapter - WholeStageCodegen (24) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (23) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #17 InputAdapter - WholeStageCodegen (43) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #17 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (91) - Project [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] #4 - 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),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_class_id,i_category_id] #21 - WholeStageCodegen (90) - 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] - SortMergeJoin [ss_item_sk,ss_item_sk] - InputAdapter - WholeStageCodegen (47) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #22 - WholeStageCodegen (46) - Filter [ss_item_sk] + BroadcastExchange #23 + WholeStageCodegen (67) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #7 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #6 - ReusedExchange [d_date_sk] #23 - InputAdapter - WholeStageCodegen (66) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #17 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (67) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #7 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - 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_item_sk,i_brand_id,i_class_id,i_category_id] #15 + 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_item_sk,i_brand_id,i_class_id,i_category_id] #15 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 ae653b43971d..5a00c3213f58 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 @@ -1,104 +1,102 @@ == Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : : : :- * HashAggregate (39) - : : : : +- Exchange (38) - : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) - : : : +- BroadcastExchange (49) - : : : +- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Filter (42) - : : : : : +- * ColumnarToRow (41) - : : : : : +- Scan parquet default.web_sales (40) - : : : : +- ReusedExchange (43) - : : : +- ReusedExchange (46) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) +TakeOrderedAndProject (98) ++- * BroadcastHashJoin Inner BuildRight (97) + :- * Filter (76) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * HashAggregate (73) + : +- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * BroadcastHashJoin LeftSemi BuildRight (57) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.item (4) + : : : +- BroadcastExchange (53) + : : : +- * HashAggregate (52) + : : : +- * HashAggregate (51) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : : : :- * HashAggregate (39) + : : : : +- Exchange (38) + : : : : +- * HashAggregate (37) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) + : : : +- BroadcastExchange (49) + : : : +- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * Project (45) + : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : :- * Filter (42) + : : : : : +- * ColumnarToRow (41) + : : : : : +- Scan parquet default.web_sales (40) + : : : : +- ReusedExchange (43) + : : : +- ReusedExchange (46) + : : +- BroadcastExchange (63) + : : +- * BroadcastHashJoin LeftSemi BuildRight (62) + : : :- * Filter (60) + : : : +- * ColumnarToRow (59) + : : : +- Scan parquet default.item (58) + : : +- ReusedExchange (61) + : +- BroadcastExchange (70) + : +- * Project (69) + : +- * Filter (68) + : +- * ColumnarToRow (67) + : +- Scan parquet default.date_dim (66) + +- BroadcastExchange (96) + +- * Filter (95) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * Project (91) + +- * BroadcastHashJoin Inner BuildRight (90) + :- * Project (84) + : +- * BroadcastHashJoin Inner BuildRight (83) + : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : :- * Filter (79) + : : : +- * ColumnarToRow (78) + : : : +- Scan parquet default.store_sales (77) + : : +- ReusedExchange (80) + : +- ReusedExchange (82) + +- BroadcastExchange (89) + +- * Project (88) + +- * Filter (87) + +- * ColumnarToRow (86) + +- Scan parquet default.date_dim (85) (1) Scan parquet default.store_sales @@ -450,277 +448,269 @@ Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Results [6]: [store AS channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#65, count(1)#63 AS number_sales#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sales#65, number_sales#66] +Condition : (isnotnull(sales#65) AND (cast(sales#65 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) -(77) Project [codegen id : 52] -Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] - -(78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +(77) Scan parquet default.store_sales +Output [4]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#72), dynamicpruningexpression(ss_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +(78) ColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72] -(80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_item_sk#70) +(79) Filter [codegen id : 50] +Input [4]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72] +Condition : isnotnull(ss_item_sk#69) -(81) ReusedExchange [Reuses operator id: unknown] +(80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#43] -(82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] +(81) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#69] Right keys [1]: [ss_item_sk#43] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +(82) ReusedExchange [Reuses operator id: 63] +Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] -(84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [i_item_sk#75] +(83) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#69] +Right keys [1]: [i_item_sk#74] Join condition: None -(85) Project [codegen id : 50] -Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +(84) Project [codegen id : 50] +Output [6]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77] +Input [8]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] -(86) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_week_seq#80] +(85) Scan parquet default.date_dim +Output [2]: [d_date_sk#78, d_week_seq#79] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] +(86) ColumnarToRow [codegen id : 49] +Input [2]: [d_date_sk#78, d_week_seq#79] -(88) Filter [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] -Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) +(87) Filter [codegen id : 49] +Input [2]: [d_date_sk#78, d_week_seq#79] +Condition : ((isnotnull(d_week_seq#79) AND (d_week_seq#79 = Subquery scalar-subquery#80, [id=#81])) AND isnotnull(d_date_sk#78)) -(89) Project [codegen id : 49] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_week_seq#80] +(88) Project [codegen id : 49] +Output [1]: [d_date_sk#78] +Input [2]: [d_date_sk#78, d_week_seq#79] -(90) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +(89) BroadcastExchange +Input [1]: [d_date_sk#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#82] -(91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#79] +(90) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#72] +Right keys [1]: [d_date_sk#78] Join condition: None -(92) Project [codegen id : 50] -Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] - -(93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] - -(94) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] - -(95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] - -(96) Filter [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) - -(97) Project [codegen id : 51] -Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] - -(98) BroadcastExchange -Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] - -(99) BroadcastHashJoin [codegen id : 52] +(91) Project [codegen id : 50] +Output [5]: [ss_quantity#70, ss_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77] +Input [7]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, d_date_sk#78] + +(92) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#70, ss_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#86, isEmpty#87, count#88] + +(93) Exchange +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, [id=#89] + +(94) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [store AS channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#93, count(1)#91 AS number_sales#94] + +(95) Filter [codegen id : 51] +Input [6]: [channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] +Condition : (isnotnull(sales#93) AND (cast(sales#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) + +(96) BroadcastExchange +Input [6]: [channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#95] + +(97) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Right keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] Join condition: None -(100) TakeOrderedAndProject -Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +(98) TakeOrderedAndProject +Input [12]: [channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sales#65, number_sales#66, channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sales#65, number_sales#66, channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] ===== Subqueries ===== Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* HashAggregate (123) -+- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- BroadcastExchange (107) - : +- * Project (106) - : +- * Filter (105) - : +- * ColumnarToRow (104) - : +- Scan parquet default.date_dim (103) - :- * Project (114) - : +- * BroadcastHashJoin Inner BuildRight (113) - : :- * ColumnarToRow (111) - : : +- Scan parquet default.catalog_sales (110) - : +- ReusedExchange (112) - +- * Project (119) - +- * BroadcastHashJoin Inner BuildRight (118) - :- * ColumnarToRow (116) - : +- Scan parquet default.web_sales (115) - +- ReusedExchange (117) - - -(101) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +* HashAggregate (121) ++- Exchange (120) + +- * HashAggregate (119) + +- Union (118) + :- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * ColumnarToRow (100) + : : +- Scan parquet default.store_sales (99) + : +- BroadcastExchange (105) + : +- * Project (104) + : +- * Filter (103) + : +- * ColumnarToRow (102) + : +- Scan parquet default.date_dim (101) + :- * Project (112) + : +- * BroadcastHashJoin Inner BuildRight (111) + : :- * ColumnarToRow (109) + : : +- Scan parquet default.catalog_sales (108) + : +- ReusedExchange (110) + +- * Project (117) + +- * BroadcastHashJoin Inner BuildRight (116) + :- * ColumnarToRow (114) + : +- Scan parquet default.web_sales (113) + +- ReusedExchange (115) + + +(99) Scan parquet default.store_sales +Output [3]: [ss_quantity#96, ss_list_price#97, ss_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +(100) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#96, ss_list_price#97, ss_sold_date_sk#98] -(103) Scan parquet default.date_dim -Output [2]: [d_date_sk#102, d_year#103] +(101) Scan parquet default.date_dim +Output [2]: [d_date_sk#100, d_year#101] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] +(102) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#100, d_year#101] -(105) Filter [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#102)) +(103) Filter [codegen id : 1] +Input [2]: [d_date_sk#100, d_year#101] +Condition : (((isnotnull(d_year#101) AND (d_year#101 >= 1999)) AND (d_year#101 <= 2001)) AND isnotnull(d_date_sk#100)) -(106) Project [codegen id : 1] -Output [1]: [d_date_sk#102] -Input [2]: [d_date_sk#102, d_year#103] +(104) Project [codegen id : 1] +Output [1]: [d_date_sk#100] +Input [2]: [d_date_sk#100, d_year#101] -(107) BroadcastExchange -Input [1]: [d_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] +(105) BroadcastExchange +Input [1]: [d_date_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] -(108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +(106) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None -(109) Project [codegen id : 2] -Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] -Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] +(107) Project [codegen id : 2] +Output [2]: [ss_quantity#96 AS quantity#103, ss_list_price#97 AS list_price#104] +Input [4]: [ss_quantity#96, ss_list_price#97, ss_sold_date_sk#98, d_date_sk#100] -(110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +(108) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#105, cs_list_price#106, cs_sold_date_sk#107] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#99)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +(109) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#105, cs_list_price#106, cs_sold_date_sk#107] -(112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#110] +(110) ReusedExchange [Reuses operator id: 105] +Output [1]: [d_date_sk#108] -(113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#109] -Right keys [1]: [d_date_sk#110] +(111) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#107] +Right keys [1]: [d_date_sk#108] Join condition: None -(114) Project [codegen id : 4] -Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] -Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] +(112) Project [codegen id : 4] +Output [2]: [cs_quantity#105 AS quantity#109, cs_list_price#106 AS list_price#110] +Input [4]: [cs_quantity#105, cs_list_price#106, cs_sold_date_sk#107, d_date_sk#108] -(115) Scan parquet default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +(113) Scan parquet default.web_sales +Output [3]: [ws_quantity#111, ws_list_price#112, ws_sold_date_sk#113] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ws_sold_date_sk#113), dynamicpruningexpression(ws_sold_date_sk#113 IN dynamicpruning#99)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +(114) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#111, ws_list_price#112, ws_sold_date_sk#113] -(117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#116] +(115) ReusedExchange [Reuses operator id: 105] +Output [1]: [d_date_sk#114] -(118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +(116) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#113] +Right keys [1]: [d_date_sk#114] Join condition: None -(119) Project [codegen id : 6] -Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] +(117) Project [codegen id : 6] +Output [2]: [ws_quantity#111 AS quantity#115, ws_list_price#112 AS list_price#116] +Input [4]: [ws_quantity#111, ws_list_price#112, ws_sold_date_sk#113, d_date_sk#114] -(120) Union +(118) Union -(121) HashAggregate [codegen id : 7] -Input [2]: [quantity#105, list_price#106] +(119) HashAggregate [codegen id : 7] +Input [2]: [quantity#103, list_price#104] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#119, count#120] -Results [2]: [sum#121, count#122] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#117, count#118] +Results [2]: [sum#119, count#120] -(122) Exchange -Input [2]: [sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +(120) Exchange +Input [2]: [sum#119, count#120] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#121] -(123) HashAggregate [codegen id : 8] -Input [2]: [sum#121, count#122] +(121) HashAggregate [codegen id : 8] +Input [2]: [sum#119, count#120] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))#122] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))#122 AS average_sales#123] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 -ReusedExchange (124) +Subquery:2 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#99 +ReusedExchange (122) -(124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#102] +(122) ReusedExchange [Reuses operator id: 105] +Output [1]: [d_date_sk#100] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 +Subquery:3 Hosting operator id = 108 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#99 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 +Subquery:4 Hosting operator id = 113 Hosting Expression = ws_sold_date_sk#113 IN dynamicpruning#99 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (125) +ReusedExchange (123) -(125) ReusedExchange [Reuses operator id: 70] +(123) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#50] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -ReusedExchange (126) +ReusedExchange (124) -(126) ReusedExchange [Reuses operator id: 26] +(124) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#29] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 @@ -728,62 +718,62 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +* Project (128) ++- * Filter (127) + +- * ColumnarToRow (126) + +- Scan parquet default.date_dim (125) -(127) Scan parquet default.date_dim -Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +(125) Scan parquet default.date_dim +Output [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] 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] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +(126) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] -(129) Filter [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 2000)) AND (d_moy#128 = 12)) AND (d_dom#129 = 11)) +(127) Filter [codegen id : 1] +Input [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] +Condition : (((((isnotnull(d_year#125) AND isnotnull(d_moy#126)) AND isnotnull(d_dom#127)) AND (d_year#125 = 2000)) AND (d_moy#126 = 12)) AND (d_dom#127 = 11)) -(130) Project [codegen id : 1] -Output [1]: [d_week_seq#126] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +(128) Project [codegen id : 1] +Output [1]: [d_week_seq#124] +Input [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +Subquery:10 Hosting operator id = 95 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 -ReusedExchange (131) +Subquery:11 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#72 IN dynamicpruning#73 +ReusedExchange (129) -(131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#79] +(129) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#78] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] -* Project (135) -+- * Filter (134) - +- * ColumnarToRow (133) - +- Scan parquet default.date_dim (132) +Subquery:12 Hosting operator id = 87 Hosting Expression = Subquery scalar-subquery#80, [id=#81] +* Project (133) ++- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet default.date_dim (130) -(132) Scan parquet default.date_dim -Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +(130) Scan parquet default.date_dim +Output [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +(131) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] -(134) Filter [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) +(132) Filter [codegen id : 1] +Input [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] +Condition : (((((isnotnull(d_year#129) AND isnotnull(d_moy#130)) AND isnotnull(d_dom#131)) AND (d_year#129 = 1999)) AND (d_moy#130 = 12)) AND (d_dom#131 = 11)) -(135) Project [codegen id : 1] -Output [1]: [d_week_seq#130] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +(133) Project [codegen id : 1] +Output [1]: [d_week_seq#128] +Input [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] 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 cb9b410ea856..bfed68434f02 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,213 +1,211 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [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 #4 - WholeStageCodegen (8) - 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 [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - ReusedExchange [d_date_sk] #14 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #14 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #14 - 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),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_class_id,i_category_id] #1 - WholeStageCodegen (25) - 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_quantity,ss_list_price,ss_sold_date_sk,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] + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + 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 [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - 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] + Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + ReusedExchange [d_date_sk] #14 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #14 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #14 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + 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_quantity,ss_list_price,ss_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + 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_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,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] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - 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_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - 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_year,d_date_sk] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + 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_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - 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] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #12 - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (24) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - 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] + 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_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + 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] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #12 + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (24) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #3 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] + ColumnarToRow + InputAdapter + 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 #15 WholeStageCodegen (51) - Project [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] #4 - 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),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_class_id,i_category_id] #16 - WholeStageCodegen (50) - 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_quantity,ss_list_price,ss_sold_date_sk,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] + Filter [sales] + ReusedSubquery [average_sales] #4 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (50) + 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_quantity,ss_list_price,ss_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #6 + ReusedExchange [d_date_sk] #17 + InputAdapter + ReusedExchange [ss_item_sk] #12 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (49) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #7 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #6 - ReusedExchange [d_date_sk] #17 - InputAdapter - ReusedExchange [ss_item_sk] #12 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (49) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #7 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - 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] + 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/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt index c61606d124cc..13638b22fdb5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt @@ -214,15 +214,15 @@ Input [4]: [_groupingexpression#19, i_item_sk#16, d_date#12, count#21] Keys [3]: [_groupingexpression#19, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS count(1)#24] +Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS cnt#24] (26) Filter [codegen id : 8] -Input [2]: [item_sk#23, count(1)#24] -Condition : (count(1)#24 > 4) +Input [2]: [item_sk#23, cnt#24] +Condition : (cnt#24 > 4) (27) Project [codegen id : 8] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#24] +Input [2]: [item_sk#23, cnt#24] (28) Sort [codegen id : 8] Input [1]: [item_sk#23] @@ -314,15 +314,15 @@ Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS ssales#38] (48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (49) Project [codegen id : 15] Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Input [2]: [c_customer_sk#31, ssales#38] (50) Sort [codegen id : 15] Input [1]: [c_customer_sk#31] @@ -338,209 +338,209 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#41, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2000)) AND (d_moy#43 = 2)) AND isnotnull(d_date_sk#41)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#41] +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] (57) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#41] Join condition: None (59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#45] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#41] (60) Scan parquet default.web_sales -Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#6)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] (62) Exchange -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Arguments: hashpartitioning(ws_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (63) Sort [codegen id : 19] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_item_sk#48 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Arguments: [ws_item_sk#46 ASC NULLS FIRST], false, 0 (64) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#54, d_date#55] +Output [2]: [ss_item_sk#52, d_date#53] (65) Sort [codegen id : 22] -Input [2]: [ss_item_sk#54, d_date#55] -Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#52, d_date#53] +Arguments: [ss_item_sk#52 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#56, i_item_desc#57] +Output [2]: [i_item_sk#54, i_item_desc#55] (67) Sort [codegen id : 24] -Input [2]: [i_item_sk#56, i_item_desc#57] -Arguments: [i_item_sk#56 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#54, i_item_desc#55] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#56] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#54] Join condition: None (69) Project [codegen id : 25] -Output [3]: [d_date#55, i_item_sk#56, substr(i_item_desc#57, 1, 30) AS _groupingexpression#58] -Input [4]: [ss_item_sk#54, d_date#55, i_item_sk#56, i_item_desc#57] +Output [3]: [d_date#53, i_item_sk#54, substr(i_item_desc#55, 1, 30) AS _groupingexpression#56] +Input [4]: [ss_item_sk#52, d_date#53, i_item_sk#54, i_item_desc#55] (70) HashAggregate [codegen id : 25] -Input [3]: [d_date#55, i_item_sk#56, _groupingexpression#58] -Keys [3]: [_groupingexpression#58, i_item_sk#56, d_date#55] +Input [3]: [d_date#53, i_item_sk#54, _groupingexpression#56] +Keys [3]: [_groupingexpression#56, i_item_sk#54, d_date#53] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#59] -Results [4]: [_groupingexpression#58, i_item_sk#56, d_date#55, count#60] +Aggregate Attributes [1]: [count#57] +Results [4]: [_groupingexpression#56, i_item_sk#54, d_date#53, count#58] (71) HashAggregate [codegen id : 25] -Input [4]: [_groupingexpression#58, i_item_sk#56, d_date#55, count#60] -Keys [3]: [_groupingexpression#58, i_item_sk#56, d_date#55] +Input [4]: [_groupingexpression#56, i_item_sk#54, d_date#53, count#58] +Keys [3]: [_groupingexpression#56, i_item_sk#54, d_date#53] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#61] -Results [2]: [i_item_sk#56 AS item_sk#23, count(1)#61 AS count(1)#62] +Aggregate Attributes [1]: [count(1)#59] +Results [2]: [i_item_sk#54 AS item_sk#23, count(1)#59 AS cnt#24] (72) Filter [codegen id : 25] -Input [2]: [item_sk#23, count(1)#62] -Condition : (count(1)#62 > 4) +Input [2]: [item_sk#23, cnt#24] +Condition : (cnt#24 > 4) (73) Project [codegen id : 25] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#62] +Input [2]: [item_sk#23, cnt#24] (74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_item_sk#48] +Left keys [1]: [ws_item_sk#46] Right keys [1]: [item_sk#23] Join condition: None (76) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Input [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] (77) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Arguments: hashpartitioning(ws_bill_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#60] (78) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Arguments: [ws_bill_customer_sk#47 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] +Output [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63] (80) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] -Arguments: [ss_customer_sk#64 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63] +Arguments: [ss_customer_sk#61 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#67] +Output [1]: [c_customer_sk#64] (82) Sort [codegen id : 31] -Input [1]: [c_customer_sk#67] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#64] +Arguments: [c_customer_sk#64 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#64] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ss_customer_sk#61] +Right keys [1]: [c_customer_sk#64] Join condition: None (84) Project [codegen id : 32] -Output [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] -Input [4]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#64] +Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, c_customer_sk#64] (85) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] -Keys [1]: [c_customer_sk#67] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] +Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#64] +Keys [1]: [c_customer_sk#64] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#63 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#64, sum#67, isEmpty#68] (86) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] -Keys [1]: [c_customer_sk#67] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72] -Results [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#64] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#63 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#63 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [2]: [c_customer_sk#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#63 as decimal(12,2)))), DecimalType(18,2), true))#69 AS ssales#38] (87) Filter [codegen id : 32] -Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#64, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (88) Project [codegen id : 32] -Output [1]: [c_customer_sk#67] -Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Output [1]: [c_customer_sk#64] +Input [2]: [c_customer_sk#64, ssales#38] (89) Sort [codegen id : 32] -Input [1]: [c_customer_sk#67] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#64] +Arguments: [c_customer_sk#64 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 34] -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#47] +Right keys [1]: [c_customer_sk#64] Join condition: None (91) Project [codegen id : 34] -Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [3]: [ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Input [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] (92) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#70] (93) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#74] +Left keys [1]: [ws_sold_date_sk#50] +Right keys [1]: [d_date_sk#70] Join condition: None (94) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#75] -Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#74] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#49 as decimal(12,2)))), DecimalType(18,2), true) AS sales#71] +Input [4]: [ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50, d_date_sk#70] (95) Union (96) HashAggregate [codegen id : 35] -Input [1]: [sales#47] +Input [1]: [sales#45] Keys: [] -Functions [1]: [partial_sum(sales#47)] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [2]: [sum#78, isEmpty#79] +Functions [1]: [partial_sum(sales#45)] +Aggregate Attributes [2]: [sum#72, isEmpty#73] +Results [2]: [sum#74, isEmpty#75] (97) Exchange -Input [2]: [sum#78, isEmpty#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#80] +Input [2]: [sum#74, isEmpty#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (98) HashAggregate [codegen id : 36] -Input [2]: [sum#78, isEmpty#79] +Input [2]: [sum#74, isEmpty#75] Keys: [] -Functions [1]: [sum(sales#47)] -Aggregate Attributes [1]: [sum(sales#47)#81] -Results [1]: [sum(sales#47)#81 AS sum(sales)#82] +Functions [1]: [sum(sales#45)] +Aggregate Attributes [1]: [sum(sales#45)#77] +Results [1]: [sum(sales#45)#77 AS sum(sales)#78] ===== Subqueries ===== @@ -549,7 +549,7 @@ ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#41] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (100) @@ -558,7 +558,7 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (124) +- Exchange (123) +- * HashAggregate (122) @@ -586,131 +586,131 @@ Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Output [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ss_sold_date_sk#82), dynamicpruningexpression(ss_sold_date_sk#82 IN dynamicpruning#83)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Input [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] -Condition : isnotnull(ss_customer_sk#83) +Input [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82] +Condition : isnotnull(ss_customer_sk#79) (104) Scan parquet default.date_dim -Output [2]: [d_date_sk#88, d_year#89] +Output [2]: [d_date_sk#84, d_year#85] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#88, d_year#89] +Input [2]: [d_date_sk#84, d_year#85] (106) Filter [codegen id : 1] -Input [2]: [d_date_sk#88, d_year#89] -Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#88)) +Input [2]: [d_date_sk#84, d_year#85] +Condition : (d_year#85 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#84)) (107) Project [codegen id : 1] -Output [1]: [d_date_sk#88] -Input [2]: [d_date_sk#88, d_year#89] +Output [1]: [d_date_sk#84] +Input [2]: [d_date_sk#84, d_year#85] (108) BroadcastExchange -Input [1]: [d_date_sk#88] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] +Input [1]: [d_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#86] (109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ss_sold_date_sk#82] +Right keys [1]: [d_date_sk#84] Join condition: None (110) Project [codegen id : 2] -Output [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, d_date_sk#88] +Output [3]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81] +Input [5]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82, d_date_sk#84] (111) Exchange -Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Arguments: hashpartitioning(ss_customer_sk#83, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [3]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81] +Arguments: hashpartitioning(ss_customer_sk#79, 5), ENSURE_REQUIREMENTS, [id=#87] (112) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Arguments: [ss_customer_sk#83 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81] +Arguments: [ss_customer_sk#79 ASC NULLS FIRST], false, 0 (113) Scan parquet default.customer -Output [1]: [c_customer_sk#92] +Output [1]: [c_customer_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#92] +Input [1]: [c_customer_sk#88] (115) Filter [codegen id : 4] -Input [1]: [c_customer_sk#92] -Condition : isnotnull(c_customer_sk#92) +Input [1]: [c_customer_sk#88] +Condition : isnotnull(c_customer_sk#88) (116) Exchange -Input [1]: [c_customer_sk#92] -Arguments: hashpartitioning(c_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [1]: [c_customer_sk#88] +Arguments: hashpartitioning(c_customer_sk#88, 5), ENSURE_REQUIREMENTS, [id=#89] (117) Sort [codegen id : 5] -Input [1]: [c_customer_sk#92] -Arguments: [c_customer_sk#92 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#88] +Arguments: [c_customer_sk#88 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#83] -Right keys [1]: [c_customer_sk#92] +Left keys [1]: [ss_customer_sk#79] +Right keys [1]: [c_customer_sk#88] Join condition: None (119) Project [codegen id : 6] -Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Output [3]: [ss_quantity#80, ss_sales_price#81, c_customer_sk#88] +Input [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, c_customer_sk#88] (120) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] -Keys [1]: [c_customer_sk#92] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#94, isEmpty#95] -Results [3]: [c_customer_sk#92, sum#96, isEmpty#97] +Input [3]: [ss_quantity#80, ss_sales_price#81, c_customer_sk#88] +Keys [1]: [c_customer_sk#88] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#80 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#81 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#90, isEmpty#91] +Results [3]: [c_customer_sk#88, sum#92, isEmpty#93] (121) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#92, sum#96, isEmpty#97] -Keys [1]: [c_customer_sk#92] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] +Input [3]: [c_customer_sk#88, sum#92, isEmpty#93] +Keys [1]: [c_customer_sk#88] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#80 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#81 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#80 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#81 as decimal(12,2)))), DecimalType(18,2), true))#94] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#80 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#81 as decimal(12,2)))), DecimalType(18,2), true))#94 AS csales#95] (122) HashAggregate [codegen id : 6] -Input [1]: [csales#99] +Input [1]: [csales#95] Keys: [] -Functions [1]: [partial_max(csales#99)] -Aggregate Attributes [1]: [max#100] -Results [1]: [max#101] +Functions [1]: [partial_max(csales#95)] +Aggregate Attributes [1]: [max#96] +Results [1]: [max#97] (123) Exchange -Input [1]: [max#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] +Input [1]: [max#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#98] (124) HashAggregate [codegen id : 7] -Input [1]: [max#101] +Input [1]: [max#97] Keys: [] -Functions [1]: [max(csales#99)] -Aggregate Attributes [1]: [max(csales#99)#103] -Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] +Functions [1]: [max(csales#95)] +Aggregate Attributes [1]: [max(csales#95)#99] +Results [1]: [max(csales#95)#99 AS tpcds_cmax#100] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#82 IN dynamicpruning#83 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#84] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] 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 0e8b0977ae08..bacae459b6cf 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 @@ -34,8 +34,8 @@ WholeStageCodegen (36) WholeStageCodegen (8) Sort [item_sk] Project [item_sk] - Filter [count(1)] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] @@ -75,7 +75,7 @@ WholeStageCodegen (36) WholeStageCodegen (15) 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))] + Filter [ssales] Subquery #3 WholeStageCodegen (7) HashAggregate [max] [max(csales),tpcds_cmax,max] @@ -119,7 +119,7 @@ WholeStageCodegen (36) ColumnarToRow InputAdapter Scan parquet default.customer [c_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,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)),ssales,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] @@ -179,8 +179,8 @@ WholeStageCodegen (36) WholeStageCodegen (25) Sort [item_sk] Project [item_sk] - Filter [count(1)] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] @@ -198,9 +198,9 @@ WholeStageCodegen (36) WholeStageCodegen (32) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 2bce3e4d9bb6..f1e533e055a6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -174,15 +174,15 @@ Input [4]: [_groupingexpression#17, i_item_sk#14, d_date#11, count#19] Keys [3]: [_groupingexpression#17, i_item_sk#14, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#21] -Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS count(1)#23] +Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS cnt#23] (22) Filter [codegen id : 4] -Input [2]: [item_sk#22, count(1)#23] -Condition : (count(1)#23 > 4) +Input [2]: [item_sk#22, cnt#23] +Condition : (cnt#23 > 4) (23) Project [codegen id : 4] Output [1]: [item_sk#22] -Input [2]: [item_sk#22, count(1)#23] +Input [2]: [item_sk#22, cnt#23] (24) BroadcastExchange Input [1]: [item_sk#22] @@ -266,15 +266,15 @@ Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS ssales#38] (42) Filter [codegen id : 9] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (43) Project [codegen id : 9] Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Input [2]: [c_customer_sk#30, ssales#38] (44) Sort [codegen id : 9] Input [1]: [c_customer_sk#30] @@ -290,128 +290,128 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (47) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#41, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] (49) Filter [codegen id : 10] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2000)) AND (d_moy#43 = 2)) AND isnotnull(d_date_sk#41)) (50) Project [codegen id : 10] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#41] +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] (51) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#45] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#41] (54) Scan parquet default.web_sales -Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#6)] ReadSchema: struct (55) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] (56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#48] +Left keys [1]: [ws_item_sk#46] Right keys [1]: [item_sk#22] Join condition: None (58) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Input [5]: [ws_item_sk#46, ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] (59) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Arguments: hashpartitioning(ws_bill_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#51] (60) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Arguments: [ws_bill_customer_sk#47 ASC NULLS FIRST], false, 0 (61) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#54, sum#55, isEmpty#56] +Output [3]: [c_customer_sk#52, sum#53, isEmpty#54] (62) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#54, sum#55, isEmpty#56] -Keys [1]: [c_customer_sk#54] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [3]: [c_customer_sk#52, sum#53, isEmpty#54] +Keys [1]: [c_customer_sk#52] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#56 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#56 as decimal(12,2)))), DecimalType(18,2), true))#57] +Results [2]: [c_customer_sk#52, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#56 as decimal(12,2)))), DecimalType(18,2), true))#57 AS ssales#38] (63) Filter [codegen id : 20] -Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#52, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (64) Project [codegen id : 20] -Output [1]: [c_customer_sk#54] -Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [1]: [c_customer_sk#52] +Input [2]: [c_customer_sk#52, ssales#38] (65) Sort [codegen id : 20] -Input [1]: [c_customer_sk#54] -Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#52] +Arguments: [c_customer_sk#52 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 22] -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#54] +Left keys [1]: [ws_bill_customer_sk#47] +Right keys [1]: [c_customer_sk#52] Join condition: None (67) Project [codegen id : 22] -Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [3]: [ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] +Input [4]: [ws_bill_customer_sk#47, ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50] (68) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#58] (69) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#61] +Left keys [1]: [ws_sold_date_sk#50] +Right keys [1]: [d_date_sk#58] Join condition: None (70) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#62] -Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#61] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#48 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#49 as decimal(12,2)))), DecimalType(18,2), true) AS sales#59] +Input [4]: [ws_quantity#48, ws_list_price#49, ws_sold_date_sk#50, d_date_sk#58] (71) Union (72) HashAggregate [codegen id : 23] -Input [1]: [sales#47] +Input [1]: [sales#45] Keys: [] -Functions [1]: [partial_sum(sales#47)] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [2]: [sum#65, isEmpty#66] +Functions [1]: [partial_sum(sales#45)] +Aggregate Attributes [2]: [sum#60, isEmpty#61] +Results [2]: [sum#62, isEmpty#63] (73) Exchange -Input [2]: [sum#65, isEmpty#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#62, isEmpty#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (74) HashAggregate [codegen id : 24] -Input [2]: [sum#65, isEmpty#66] +Input [2]: [sum#62, isEmpty#63] Keys: [] -Functions [1]: [sum(sales#47)] -Aggregate Attributes [1]: [sum(sales#47)#68] -Results [1]: [sum(sales#47)#68 AS sum(sales)#69] +Functions [1]: [sum(sales#45)] +Aggregate Attributes [1]: [sum(sales#45)#65] +Results [1]: [sum(sales#45)#65 AS sum(sales)#66] ===== Subqueries ===== @@ -420,7 +420,7 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#41] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (76) @@ -429,7 +429,7 @@ ReusedExchange (76) (76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (98) +- Exchange (97) +- * HashAggregate (96) @@ -455,123 +455,123 @@ Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Output [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_customer_sk#70) +Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] +Condition : isnotnull(ss_customer_sk#67) (80) Scan parquet default.customer -Output [1]: [c_customer_sk#75] +Output [1]: [c_customer_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#75] +Input [1]: [c_customer_sk#72] (82) Filter [codegen id : 1] -Input [1]: [c_customer_sk#75] -Condition : isnotnull(c_customer_sk#75) +Input [1]: [c_customer_sk#72] +Condition : isnotnull(c_customer_sk#72) (83) BroadcastExchange -Input [1]: [c_customer_sk#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Input [1]: [c_customer_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#70] -Right keys [1]: [c_customer_sk#75] +Left keys [1]: [ss_customer_sk#67] +Right keys [1]: [c_customer_sk#72] Join condition: None (85) Project [codegen id : 3] -Output [4]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] -Input [5]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] +Output [4]: [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72] +Input [5]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#74, d_year#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#77, d_year#78] +Input [2]: [d_date_sk#74, d_year#75] (88) Filter [codegen id : 2] -Input [2]: [d_date_sk#77, d_year#78] -Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#77)) +Input [2]: [d_date_sk#74, d_year#75] +Condition : (d_year#75 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#74)) (89) Project [codegen id : 2] -Output [1]: [d_date_sk#77] -Input [2]: [d_date_sk#77, d_year#78] +Output [1]: [d_date_sk#74] +Input [2]: [d_date_sk#74, d_year#75] (90) BroadcastExchange -Input [1]: [d_date_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] +Input [1]: [d_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (91) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ss_sold_date_sk#70] +Right keys [1]: [d_date_sk#74] Join condition: None (92) Project [codegen id : 3] -Output [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] -Input [5]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75, d_date_sk#77] +Output [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] +Input [5]: [ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70, c_customer_sk#72, d_date_sk#74] (93) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] -Keys [1]: [c_customer_sk#75] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#80, isEmpty#81] -Results [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Input [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] +Keys [1]: [c_customer_sk#72] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#77, isEmpty#78] +Results [3]: [c_customer_sk#72, sum#79, isEmpty#80] (94) Exchange -Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] -Arguments: hashpartitioning(c_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [3]: [c_customer_sk#72, sum#79, isEmpty#80] +Arguments: hashpartitioning(c_customer_sk#72, 5), ENSURE_REQUIREMENTS, [id=#81] (95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] -Keys [1]: [c_customer_sk#75] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85 AS csales#86] +Input [3]: [c_customer_sk#72, sum#79, isEmpty#80] +Keys [1]: [c_customer_sk#72] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 AS csales#83] (96) HashAggregate [codegen id : 4] -Input [1]: [csales#86] +Input [1]: [csales#83] Keys: [] -Functions [1]: [partial_max(csales#86)] -Aggregate Attributes [1]: [max#87] -Results [1]: [max#88] +Functions [1]: [partial_max(csales#83)] +Aggregate Attributes [1]: [max#84] +Results [1]: [max#85] (97) Exchange -Input [1]: [max#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] +Input [1]: [max#85] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#86] (98) HashAggregate [codegen id : 5] -Input [1]: [max#88] +Input [1]: [max#85] Keys: [] -Functions [1]: [max(csales#86)] -Aggregate Attributes [1]: [max(csales#86)#90] -Results [1]: [max(csales#86)#90 AS tpcds_cmax#91] +Functions [1]: [max(csales#83)] +Aggregate Attributes [1]: [max(csales#83)#87] +Results [1]: [max(csales#83)#87 AS tpcds_cmax#88] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#74] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] 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 697c1f497c39..f0dfeaac0c56 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 @@ -28,8 +28,8 @@ WholeStageCodegen (24) BroadcastExchange #4 WholeStageCodegen (4) Project [item_sk] - Filter [count(1)] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] InputAdapter Exchange [_groupingexpression,i_item_sk,d_date] #5 WholeStageCodegen (3) @@ -63,7 +63,7 @@ WholeStageCodegen (24) WholeStageCodegen (9) 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))] + Filter [ssales] Subquery #3 WholeStageCodegen (5) HashAggregate [max] [max(csales),tpcds_cmax,max] @@ -101,7 +101,7 @@ WholeStageCodegen (24) ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - 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,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)),ssales,sum,isEmpty] InputAdapter Exchange [c_customer_sk] #8 WholeStageCodegen (8) @@ -151,9 +151,9 @@ WholeStageCodegen (24) WholeStageCodegen (20) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,sum,isEmpty] InputAdapter ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index c5b8ad460755..49f575cf32f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -258,15 +258,15 @@ Input [4]: [_groupingexpression#19, i_item_sk#16, d_date#12, count#21] Keys [3]: [_groupingexpression#19, i_item_sk#16, d_date#12] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS count(1)#24] +Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#22 AS cnt#24] (27) Filter [codegen id : 8] -Input [2]: [item_sk#23, count(1)#24] -Condition : (count(1)#24 > 4) +Input [2]: [item_sk#23, cnt#24] +Condition : (cnt#24 > 4) (28) Project [codegen id : 8] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#24] +Input [2]: [item_sk#23, cnt#24] (29) Sort [codegen id : 8] Input [1]: [item_sk#23] @@ -358,15 +358,15 @@ Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS ssales#38] (49) Filter [codegen id : 15] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (50) Project [codegen id : 15] Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Input [2]: [c_customer_sk#31, ssales#38] (51) Sort [codegen id : 15] Input [1]: [c_customer_sk#31] @@ -378,57 +378,57 @@ Right keys [1]: [c_customer_sk#31] Join condition: None (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#41, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2000)) AND (d_moy#43 = 2)) AND isnotnull(d_date_sk#41)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#41] +Input [3]: [d_date_sk#41, d_year#42, d_moy#43] (57) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#41] Join condition: None (59) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#41] (60) Scan parquet default.customer -Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Input [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] (62) Filter [codegen id : 18] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Condition : isnotnull(c_customer_sk#47) +Input [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] +Condition : isnotnull(c_customer_sk#45) (63) Exchange -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Arguments: hashpartitioning(c_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] +Arguments: hashpartitioning(c_customer_sk#45, 5), ENSURE_REQUIREMENTS, [id=#48] (64) Sort [codegen id : 19] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] +Arguments: [c_customer_sk#45 ASC NULLS FIRST], false, 0 (65) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] @@ -465,300 +465,300 @@ Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS ssales#38] (73) Filter [codegen id : 24] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 24] Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Input [2]: [c_customer_sk#31, ssales#38] (75) Sort [codegen id : 24] Input [1]: [c_customer_sk#31] Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (76) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_customer_sk#47] +Left keys [1]: [c_customer_sk#45] Right keys [1]: [c_customer_sk#31] Join condition: None (77) SortMergeJoin [codegen id : 26] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#45] Join condition: None (78) Project [codegen id : 26] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#46, c_last_name#47] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#45, c_first_name#46, c_last_name#47] (79) HashAggregate [codegen id : 26] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] -Keys [2]: [c_last_name#49, c_first_name#48] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#46, c_last_name#47] +Keys [2]: [c_last_name#47, c_first_name#46] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] (80) Exchange -Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] +Arguments: hashpartitioning(c_last_name#47, c_first_name#46, 5), ENSURE_REQUIREMENTS, [id=#53] (81) HashAggregate [codegen id : 27] -Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -Keys [2]: [c_last_name#49, c_first_name#48] +Input [4]: [c_last_name#47, c_first_name#46, sum#51, isEmpty#52] +Keys [2]: [c_last_name#47, c_first_name#46] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56] -Results [3]: [c_last_name#49, c_first_name#48, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#57] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#54] +Results [3]: [c_last_name#47, c_first_name#46, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#55] (82) Scan parquet default.web_sales -Output [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Output [5]: [ws_item_sk#56, ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 28] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Input [5]: [ws_item_sk#56, ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] (84) Filter [codegen id : 28] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [5]: [ws_item_sk#56, ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_customer_sk#57) (85) Exchange -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [5]: [ws_item_sk#56, ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] +Arguments: hashpartitioning(ws_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#61] (86) Sort [codegen id : 29] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#56, ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] +Arguments: [ws_item_sk#56 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#64, d_date#65] +Output [2]: [ss_item_sk#62, d_date#63] (88) Sort [codegen id : 32] -Input [2]: [ss_item_sk#64, d_date#65] -Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#62, d_date#63] +Arguments: [ss_item_sk#62 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#66, i_item_desc#67] +Output [2]: [i_item_sk#64, i_item_desc#65] (90) Sort [codegen id : 34] -Input [2]: [i_item_sk#66, i_item_desc#67] -Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#64, i_item_desc#65] +Arguments: [i_item_sk#64 ASC NULLS FIRST], false, 0 (91) SortMergeJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#64] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#62] +Right keys [1]: [i_item_sk#64] Join condition: None (92) Project [codegen id : 35] -Output [3]: [d_date#65, i_item_sk#66, substr(i_item_desc#67, 1, 30) AS _groupingexpression#68] -Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] +Output [3]: [d_date#63, i_item_sk#64, substr(i_item_desc#65, 1, 30) AS _groupingexpression#66] +Input [4]: [ss_item_sk#62, d_date#63, i_item_sk#64, i_item_desc#65] (93) HashAggregate [codegen id : 35] -Input [3]: [d_date#65, i_item_sk#66, _groupingexpression#68] -Keys [3]: [_groupingexpression#68, i_item_sk#66, d_date#65] +Input [3]: [d_date#63, i_item_sk#64, _groupingexpression#66] +Keys [3]: [_groupingexpression#66, i_item_sk#64, d_date#63] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#69] -Results [4]: [_groupingexpression#68, i_item_sk#66, d_date#65, count#70] +Aggregate Attributes [1]: [count#67] +Results [4]: [_groupingexpression#66, i_item_sk#64, d_date#63, count#68] (94) HashAggregate [codegen id : 35] -Input [4]: [_groupingexpression#68, i_item_sk#66, d_date#65, count#70] -Keys [3]: [_groupingexpression#68, i_item_sk#66, d_date#65] +Input [4]: [_groupingexpression#66, i_item_sk#64, d_date#63, count#68] +Keys [3]: [_groupingexpression#66, i_item_sk#64, d_date#63] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#71] -Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] +Aggregate Attributes [1]: [count(1)#69] +Results [2]: [i_item_sk#64 AS item_sk#23, count(1)#69 AS cnt#24] (95) Filter [codegen id : 35] -Input [2]: [item_sk#23, count(1)#72] -Condition : (count(1)#72 > 4) +Input [2]: [item_sk#23, cnt#24] +Condition : (cnt#24 > 4) (96) Project [codegen id : 35] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#72] +Input [2]: [item_sk#23, cnt#24] (97) Sort [codegen id : 35] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 36] -Left keys [1]: [ws_item_sk#58] +Left keys [1]: [ws_item_sk#56] Right keys [1]: [item_sk#23] Join condition: None (99) Project [codegen id : 36] -Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Output [4]: [ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] +Input [5]: [ws_item_sk#56, ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] (100) Exchange -Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] +Arguments: hashpartitioning(ws_bill_customer_sk#57, 5), ENSURE_REQUIREMENTS, [id=#70] (101) Sort [codegen id : 37] -Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60] +Arguments: [ws_bill_customer_sk#57 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Output [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] (103) Sort [codegen id : 39] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] +Arguments: [ss_customer_sk#71 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#77] +Output [1]: [c_customer_sk#74] (105) Sort [codegen id : 41] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#74] +Arguments: [c_customer_sk#74 ASC NULLS FIRST], false, 0 (106) SortMergeJoin [codegen id : 42] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ss_customer_sk#71] +Right keys [1]: [c_customer_sk#74] Join condition: None (107) Project [codegen id : 42] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Output [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#74] +Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, c_customer_sk#74] (108) HashAggregate [codegen id : 42] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Keys [1]: [c_customer_sk#77] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Input [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#74] +Keys [1]: [c_customer_sk#74] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#75, isEmpty#76] +Results [3]: [c_customer_sk#74, sum#77, isEmpty#78] (109) HashAggregate [codegen id : 42] -Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#77] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Input [3]: [c_customer_sk#74, sum#77, isEmpty#78] +Keys [1]: [c_customer_sk#74] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79] +Results [2]: [c_customer_sk#74, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79 AS ssales#38] (110) Filter [codegen id : 42] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#74, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (111) Project [codegen id : 42] -Output [1]: [c_customer_sk#77] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Output [1]: [c_customer_sk#74] +Input [2]: [c_customer_sk#74, ssales#38] (112) Sort [codegen id : 42] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#74] +Arguments: [c_customer_sk#74 ASC NULLS FIRST], false, 0 (113) SortMergeJoin [codegen id : 44] -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ws_bill_customer_sk#57] +Right keys [1]: [c_customer_sk#74] Join condition: None (114) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#84] +Output [1]: [d_date_sk#80] (115) BroadcastHashJoin [codegen id : 44] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#80] Join condition: None (116) Project [codegen id : 44] -Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] -Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] +Output [3]: [ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59] +Input [5]: [ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, ws_sold_date_sk#60, d_date_sk#80] (117) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] +Output [3]: [c_customer_sk#81, c_first_name#82, c_last_name#83] (118) Sort [codegen id : 46] -Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] -Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#81, c_first_name#82, c_last_name#83] +Arguments: [c_customer_sk#81 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Output [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] (120) Sort [codegen id : 48] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73] +Arguments: [ss_customer_sk#71 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#77] +Output [1]: [c_customer_sk#74] (122) Sort [codegen id : 50] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#74] +Arguments: [c_customer_sk#74 ASC NULLS FIRST], false, 0 (123) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ss_customer_sk#71] +Right keys [1]: [c_customer_sk#74] Join condition: None (124) Project [codegen id : 51] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Output [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#74] +Input [4]: [ss_customer_sk#71, ss_quantity#72, ss_sales_price#73, c_customer_sk#74] (125) HashAggregate [codegen id : 51] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Keys [1]: [c_customer_sk#77] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Input [3]: [ss_quantity#72, ss_sales_price#73, c_customer_sk#74] +Keys [1]: [c_customer_sk#74] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#75, isEmpty#76] +Results [3]: [c_customer_sk#74, sum#77, isEmpty#78] (126) HashAggregate [codegen id : 51] -Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#77] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Input [3]: [c_customer_sk#74, sum#77, isEmpty#78] +Keys [1]: [c_customer_sk#74] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79] +Results [2]: [c_customer_sk#74, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#72 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#73 as decimal(12,2)))), DecimalType(18,2), true))#79 AS ssales#38] (127) Filter [codegen id : 51] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#74, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (128) Project [codegen id : 51] -Output [1]: [c_customer_sk#77] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Output [1]: [c_customer_sk#74] +Input [2]: [c_customer_sk#74, ssales#38] (129) Sort [codegen id : 51] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#74] +Arguments: [c_customer_sk#74 ASC NULLS FIRST], false, 0 (130) SortMergeJoin [codegen id : 52] -Left keys [1]: [c_customer_sk#85] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [c_customer_sk#81] +Right keys [1]: [c_customer_sk#74] Join condition: None (131) SortMergeJoin [codegen id : 53] -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#85] +Left keys [1]: [ws_bill_customer_sk#57] +Right keys [1]: [c_customer_sk#81] Join condition: None (132) Project [codegen id : 53] -Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] -Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] +Output [4]: [ws_quantity#58, ws_list_price#59, c_first_name#82, c_last_name#83] +Input [6]: [ws_bill_customer_sk#57, ws_quantity#58, ws_list_price#59, c_customer_sk#81, c_first_name#82, c_last_name#83] (133) HashAggregate [codegen id : 53] -Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] -Keys [2]: [c_last_name#87, c_first_name#86] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#88, isEmpty#89] -Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Input [4]: [ws_quantity#58, ws_list_price#59, c_first_name#82, c_last_name#83] +Keys [2]: [c_last_name#83, c_first_name#82] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#58 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#59 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#84, isEmpty#85] +Results [4]: [c_last_name#83, c_first_name#82, sum#86, isEmpty#87] (134) Exchange -Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] -Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [c_last_name#83, c_first_name#82, sum#86, isEmpty#87] +Arguments: hashpartitioning(c_last_name#83, c_first_name#82, 5), ENSURE_REQUIREMENTS, [id=#88] (135) HashAggregate [codegen id : 54] -Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] -Keys [2]: [c_last_name#87, c_first_name#86] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93] -Results [3]: [c_last_name#87, c_first_name#86, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93 AS sales#94] +Input [4]: [c_last_name#83, c_first_name#82, sum#86, isEmpty#87] +Keys [2]: [c_last_name#83, c_first_name#82] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#58 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#59 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#58 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#59 as decimal(12,2)))), DecimalType(18,2), true))#89] +Results [3]: [c_last_name#83, c_first_name#82, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#58 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#59 as decimal(12,2)))), DecimalType(18,2), true))#89 AS sales#90] (136) Union (137) TakeOrderedAndProject -Input [3]: [c_last_name#49, c_first_name#48, sales#57] -Arguments: 100, [c_last_name#49 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, sales#57 ASC NULLS FIRST], [c_last_name#49, c_first_name#48, sales#57] +Input [3]: [c_last_name#47, c_first_name#46, sales#55] +Arguments: 100, [c_last_name#47 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, sales#55 ASC NULLS FIRST], [c_last_name#47, c_first_name#46, sales#55] ===== Subqueries ===== @@ -767,7 +767,7 @@ ReusedExchange (138) (138) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#41] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (139) @@ -776,7 +776,7 @@ ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (163) +- Exchange (162) +- * HashAggregate (161) @@ -804,135 +804,135 @@ Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Output [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] +PartitionFilters: [isnotnull(ss_sold_date_sk#94), dynamicpruningexpression(ss_sold_date_sk#94 IN dynamicpruning#95)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Input [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] -Condition : isnotnull(ss_customer_sk#95) +Input [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94] +Condition : isnotnull(ss_customer_sk#91) (143) Scan parquet default.date_dim -Output [2]: [d_date_sk#100, d_year#101] +Output [2]: [d_date_sk#96, d_year#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#100, d_year#101] +Input [2]: [d_date_sk#96, d_year#97] (145) Filter [codegen id : 1] -Input [2]: [d_date_sk#100, d_year#101] -Condition : (d_year#101 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#100)) +Input [2]: [d_date_sk#96, d_year#97] +Condition : (d_year#97 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#96)) (146) Project [codegen id : 1] -Output [1]: [d_date_sk#100] -Input [2]: [d_date_sk#100, d_year#101] +Output [1]: [d_date_sk#96] +Input [2]: [d_date_sk#96, d_year#97] (147) BroadcastExchange -Input [1]: [d_date_sk#100] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] +Input [1]: [d_date_sk#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (148) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#98] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ss_sold_date_sk#94] +Right keys [1]: [d_date_sk#96] Join condition: None (149) Project [codegen id : 2] -Output [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Input [5]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98, d_date_sk#100] +Output [3]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93] +Input [5]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94, d_date_sk#96] (150) Exchange -Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Arguments: hashpartitioning(ss_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [3]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93] +Arguments: hashpartitioning(ss_customer_sk#91, 5), ENSURE_REQUIREMENTS, [id=#99] (151) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Arguments: [ss_customer_sk#95 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93] +Arguments: [ss_customer_sk#91 ASC NULLS FIRST], false, 0 (152) Scan parquet default.customer -Output [1]: [c_customer_sk#104] +Output [1]: [c_customer_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#104] +Input [1]: [c_customer_sk#100] (154) Filter [codegen id : 4] -Input [1]: [c_customer_sk#104] -Condition : isnotnull(c_customer_sk#104) +Input [1]: [c_customer_sk#100] +Condition : isnotnull(c_customer_sk#100) (155) Exchange -Input [1]: [c_customer_sk#104] -Arguments: hashpartitioning(c_customer_sk#104, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [1]: [c_customer_sk#100] +Arguments: hashpartitioning(c_customer_sk#100, 5), ENSURE_REQUIREMENTS, [id=#101] (156) Sort [codegen id : 5] -Input [1]: [c_customer_sk#104] -Arguments: [c_customer_sk#104 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#100] +Arguments: [c_customer_sk#100 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#95] -Right keys [1]: [c_customer_sk#104] +Left keys [1]: [ss_customer_sk#91] +Right keys [1]: [c_customer_sk#100] Join condition: None (158) Project [codegen id : 6] -Output [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Output [3]: [ss_quantity#92, ss_sales_price#93, c_customer_sk#100] +Input [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, c_customer_sk#100] (159) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] -Keys [1]: [c_customer_sk#104] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, isEmpty#107] -Results [3]: [c_customer_sk#104, sum#108, isEmpty#109] +Input [3]: [ss_quantity#92, ss_sales_price#93, c_customer_sk#100] +Keys [1]: [c_customer_sk#100] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#93 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#102, isEmpty#103] +Results [3]: [c_customer_sk#100, sum#104, isEmpty#105] (160) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#104, sum#108, isEmpty#109] -Keys [1]: [c_customer_sk#104] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110 AS csales#111] +Input [3]: [c_customer_sk#100, sum#104, isEmpty#105] +Keys [1]: [c_customer_sk#100] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#93 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#93 as decimal(12,2)))), DecimalType(18,2), true))#106] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#93 as decimal(12,2)))), DecimalType(18,2), true))#106 AS csales#107] (161) HashAggregate [codegen id : 6] -Input [1]: [csales#111] +Input [1]: [csales#107] Keys: [] -Functions [1]: [partial_max(csales#111)] -Aggregate Attributes [1]: [max#112] -Results [1]: [max#113] +Functions [1]: [partial_max(csales#107)] +Aggregate Attributes [1]: [max#108] +Results [1]: [max#109] (162) Exchange -Input [1]: [max#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [1]: [max#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (163) HashAggregate [codegen id : 7] -Input [1]: [max#113] +Input [1]: [max#109] Keys: [] -Functions [1]: [max(csales#111)] -Aggregate Attributes [1]: [max(csales#111)#115] -Results [1]: [max(csales#111)#115 AS tpcds_cmax#116] +Functions [1]: [max(csales#107)] +Aggregate Attributes [1]: [max(csales#107)#111] +Results [1]: [max(csales#107)#111 AS tpcds_cmax#112] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#99 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#94 IN dynamicpruning#95 ReusedExchange (164) (164) ReusedExchange [Reuses operator id: 147] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#96] -Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] -Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] -Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] 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 98848b4ed067..9e0dd47ae5c7 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 @@ -37,8 +37,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (8) Sort [item_sk] Project [item_sk] - Filter [count(1)] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] @@ -78,7 +78,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (15) 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))] + Filter [ssales] Subquery #3 WholeStageCodegen (7) HashAggregate [max] [max(csales),tpcds_cmax,max] @@ -122,7 +122,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter Scan parquet default.customer [c_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,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)),ssales,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] @@ -172,9 +172,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (24) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,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] @@ -224,8 +224,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (35) Sort [item_sk] Project [item_sk] - Filter [count(1)] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] Project [d_date,i_item_sk,i_item_desc] SortMergeJoin [ss_item_sk,i_item_sk] @@ -243,9 +243,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (42) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,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] @@ -273,9 +273,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (51) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 2eb4beb63497..116570c8e0dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -200,15 +200,15 @@ Input [4]: [_groupingexpression#17, i_item_sk#14, d_date#11, count#19] Keys [3]: [_groupingexpression#17, i_item_sk#14, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#21] -Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS count(1)#23] +Results [2]: [i_item_sk#14 AS item_sk#22, count(1)#21 AS cnt#23] (23) Filter [codegen id : 4] -Input [2]: [item_sk#22, count(1)#23] -Condition : (count(1)#23 > 4) +Input [2]: [item_sk#22, cnt#23] +Condition : (cnt#23 > 4) (24) Project [codegen id : 4] Output [1]: [item_sk#22] -Input [2]: [item_sk#22, count(1)#23] +Input [2]: [item_sk#22, cnt#23] (25) BroadcastExchange Input [1]: [item_sk#22] @@ -292,15 +292,15 @@ Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS ssales#38] (43) Filter [codegen id : 9] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (44) Project [codegen id : 9] Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Input [2]: [c_customer_sk#30, ssales#38] (45) Sort [codegen id : 9] Input [1]: [c_customer_sk#30] @@ -312,26 +312,26 @@ Right keys [1]: [c_customer_sk#30] Join condition: None (47) Scan parquet default.customer -Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Output [3]: [c_customer_sk#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Input [3]: [c_customer_sk#41, c_first_name#42, c_last_name#43] (49) Filter [codegen id : 10] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Condition : isnotnull(c_customer_sk#43) +Input [3]: [c_customer_sk#41, c_first_name#42, c_last_name#43] +Condition : isnotnull(c_customer_sk#41) (50) Exchange -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: hashpartitioning(c_customer_sk#43, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [c_customer_sk#41, c_first_name#42, c_last_name#43] +Arguments: hashpartitioning(c_customer_sk#41, 5), ENSURE_REQUIREMENTS, [id=#44] (51) Sort [codegen id : 11] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: [c_customer_sk#43 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#41, c_first_name#42, c_last_name#43] +Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 (52) ReusedExchange [Reuses operator id: 41] Output [3]: [c_customer_sk#30, sum#34, isEmpty#35] @@ -341,196 +341,196 @@ Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS ssales#38] (54) Filter [codegen id : 14] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (55) Project [codegen id : 14] Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Input [2]: [c_customer_sk#30, ssales#38] (56) Sort [codegen id : 14] Input [1]: [c_customer_sk#30] Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (57) SortMergeJoin [codegen id : 15] -Left keys [1]: [c_customer_sk#43] +Left keys [1]: [c_customer_sk#41] Right keys [1]: [c_customer_sk#30] Join condition: None (58) BroadcastExchange -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [c_customer_sk#41, c_first_name#42, c_last_name#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] (59) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#43] +Right keys [1]: [c_customer_sk#41] Join condition: None (60) Project [codegen id : 17] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#42, c_last_name#43] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#41, c_first_name#42, c_last_name#43] (61) Scan parquet default.date_dim -Output [3]: [d_date_sk#48, d_year#49, d_moy#50] +Output [3]: [d_date_sk#46, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Input [3]: [d_date_sk#46, d_year#47, d_moy#48] (63) Filter [codegen id : 16] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) +Input [3]: [d_date_sk#46, d_year#47, d_moy#48] +Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 2000)) AND (d_moy#48 = 2)) AND isnotnull(d_date_sk#46)) (64) Project [codegen id : 16] -Output [1]: [d_date_sk#48] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Output [1]: [d_date_sk#46] +Input [3]: [d_date_sk#46, d_year#47, d_moy#48] (65) BroadcastExchange -Input [1]: [d_date_sk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [1]: [d_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (66) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#48] +Right keys [1]: [d_date_sk#46] Join condition: None (67) Project [codegen id : 17] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#42, c_last_name#43, d_date_sk#46] (68) HashAggregate [codegen id : 17] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] -Keys [2]: [c_last_name#45, c_first_name#44] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] +Keys [2]: [c_last_name#43, c_first_name#42] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [4]: [c_last_name#43, c_first_name#42, sum#52, isEmpty#53] (69) Exchange -Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] -Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [c_last_name#43, c_first_name#42, sum#52, isEmpty#53] +Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#54] (70) HashAggregate [codegen id : 18] -Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] -Keys [2]: [c_last_name#45, c_first_name#44] +Input [4]: [c_last_name#43, c_first_name#42, sum#52, isEmpty#53] +Keys [2]: [c_last_name#43, c_first_name#42] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57] -Results [3]: [c_last_name#45, c_first_name#44, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sales#58] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#55] +Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#56] (71) Scan parquet default.web_sales -Output [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Output [5]: [ws_item_sk#57, ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 23] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Input [5]: [ws_item_sk#57, ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] (73) Filter [codegen id : 23] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) +Input [5]: [ws_item_sk#57, ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_bill_customer_sk#58) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (75) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ws_item_sk#59] +Left keys [1]: [ws_item_sk#57] Right keys [1]: [item_sk#22] Join condition: None (76) Project [codegen id : 23] -Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Output [4]: [ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] +Input [5]: [ws_item_sk#57, ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] (77) Exchange -Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] +Arguments: hashpartitioning(ws_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] (78) Sort [codegen id : 24] -Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61] +Arguments: [ws_bill_customer_sk#58 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Output [3]: [c_customer_sk#63, sum#64, isEmpty#65] (80) HashAggregate [codegen id : 27] -Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Input [3]: [c_customer_sk#63, sum#64, isEmpty#65] +Keys [1]: [c_customer_sk#63] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2), true))#68] +Results [2]: [c_customer_sk#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2), true))#68 AS ssales#38] (81) Filter [codegen id : 27] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#63, ssales#38] +Condition : (isnotnull(ssales#38) AND (cast(ssales#38 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#39, [id=#40] as decimal(32,6)))), DecimalType(38,8), true))) (82) Project [codegen id : 27] -Output [1]: [c_customer_sk#65] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [1]: [c_customer_sk#63] +Input [2]: [c_customer_sk#63, ssales#38] (83) Sort [codegen id : 27] -Input [1]: [c_customer_sk#65] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#63] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [ws_bill_customer_sk#58] +Right keys [1]: [c_customer_sk#63] Join condition: None (85) ReusedExchange [Reuses operator id: unknown] -Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Output [3]: [c_customer_sk#69, c_first_name#70, c_last_name#71] (86) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#72] +Left keys [1]: [ws_bill_customer_sk#58] +Right keys [1]: [c_customer_sk#69] Join condition: None (87) Project [codegen id : 35] -Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] -Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] +Output [5]: [ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61, c_first_name#70, c_last_name#71] +Input [7]: [ws_bill_customer_sk#58, ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61, c_customer_sk#69, c_first_name#70, c_last_name#71] (88) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#75] +Output [1]: [d_date_sk#72] (89) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#75] +Left keys [1]: [ws_sold_date_sk#61] +Right keys [1]: [d_date_sk#72] Join condition: None (90) Project [codegen id : 35] -Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#75] +Output [4]: [ws_quantity#59, ws_list_price#60, c_first_name#70, c_last_name#71] +Input [6]: [ws_quantity#59, ws_list_price#60, ws_sold_date_sk#61, c_first_name#70, c_last_name#71, d_date_sk#72] (91) HashAggregate [codegen id : 35] -Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Keys [2]: [c_last_name#74, c_first_name#73] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79] +Input [4]: [ws_quantity#59, ws_list_price#60, c_first_name#70, c_last_name#71] +Keys [2]: [c_last_name#71, c_first_name#70] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#59 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#60 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#73, isEmpty#74] +Results [4]: [c_last_name#71, c_first_name#70, sum#75, isEmpty#76] (92) Exchange -Input [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [c_last_name#71, c_first_name#70, sum#75, isEmpty#76] +Arguments: hashpartitioning(c_last_name#71, c_first_name#70, 5), ENSURE_REQUIREMENTS, [id=#77] (93) HashAggregate [codegen id : 36] -Input [4]: [c_last_name#74, c_first_name#73, sum#78, isEmpty#79] -Keys [2]: [c_last_name#74, c_first_name#73] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#81] -Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#82] +Input [4]: [c_last_name#71, c_first_name#70, sum#75, isEmpty#76] +Keys [2]: [c_last_name#71, c_first_name#70] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#59 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#60 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#59 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#60 as decimal(12,2)))), DecimalType(18,2), true))#78] +Results [3]: [c_last_name#71, c_first_name#70, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#59 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#60 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#79] (94) Union (95) TakeOrderedAndProject -Input [3]: [c_last_name#45, c_first_name#44, sales#58] -Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] +Input [3]: [c_last_name#43, c_first_name#42, sales#56] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, sales#56 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, sales#56] ===== Subqueries ===== @@ -539,7 +539,7 @@ ReusedExchange (96) (96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#48] +Output [1]: [d_date_sk#46] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (97) @@ -548,7 +548,7 @@ ReusedExchange (97) (97) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (119) +- Exchange (118) +- * HashAggregate (117) @@ -574,125 +574,125 @@ Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (98) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Output [4]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (99) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Input [4]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83] (100) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] -Condition : isnotnull(ss_customer_sk#83) +Input [4]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83] +Condition : isnotnull(ss_customer_sk#80) (101) Scan parquet default.customer -Output [1]: [c_customer_sk#88] +Output [1]: [c_customer_sk#85] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#88] +Input [1]: [c_customer_sk#85] (103) Filter [codegen id : 1] -Input [1]: [c_customer_sk#88] -Condition : isnotnull(c_customer_sk#88) +Input [1]: [c_customer_sk#85] +Condition : isnotnull(c_customer_sk#85) (104) BroadcastExchange -Input [1]: [c_customer_sk#88] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#89] +Input [1]: [c_customer_sk#85] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (105) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#83] -Right keys [1]: [c_customer_sk#88] +Left keys [1]: [ss_customer_sk#80] +Right keys [1]: [c_customer_sk#85] Join condition: None (106) Project [codegen id : 3] -Output [4]: [ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88] -Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88] +Output [4]: [ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83, c_customer_sk#85] +Input [5]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83, c_customer_sk#85] (107) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#87, d_year#88] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#87, d_year#88] (109) Filter [codegen id : 2] -Input [2]: [d_date_sk#90, d_year#91] -Condition : (d_year#91 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#87, d_year#88] +Condition : (d_year#88 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#87)) (110) Project [codegen id : 2] -Output [1]: [d_date_sk#90] -Input [2]: [d_date_sk#90, d_year#91] +Output [1]: [d_date_sk#87] +Input [2]: [d_date_sk#87, d_year#88] (111) BroadcastExchange -Input [1]: [d_date_sk#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] +Input [1]: [d_date_sk#87] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#89] (112) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [ss_sold_date_sk#83] +Right keys [1]: [d_date_sk#87] Join condition: None (113) Project [codegen id : 3] -Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#88] -Input [5]: [ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, c_customer_sk#88, d_date_sk#90] +Output [3]: [ss_quantity#81, ss_sales_price#82, c_customer_sk#85] +Input [5]: [ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83, c_customer_sk#85, d_date_sk#87] (114) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#88] -Keys [1]: [c_customer_sk#88] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [3]: [c_customer_sk#88, sum#95, isEmpty#96] +Input [3]: [ss_quantity#81, ss_sales_price#82, c_customer_sk#85] +Keys [1]: [c_customer_sk#85] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#81 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#82 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#90, isEmpty#91] +Results [3]: [c_customer_sk#85, sum#92, isEmpty#93] (115) Exchange -Input [3]: [c_customer_sk#88, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_sk#88, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [3]: [c_customer_sk#85, sum#92, isEmpty#93] +Arguments: hashpartitioning(c_customer_sk#85, 5), ENSURE_REQUIREMENTS, [id=#94] (116) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#88, sum#95, isEmpty#96] -Keys [1]: [c_customer_sk#88] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] +Input [3]: [c_customer_sk#85, sum#92, isEmpty#93] +Keys [1]: [c_customer_sk#85] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#81 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#82 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#81 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#82 as decimal(12,2)))), DecimalType(18,2), true))#95] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#81 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#82 as decimal(12,2)))), DecimalType(18,2), true))#95 AS csales#96] (117) HashAggregate [codegen id : 4] -Input [1]: [csales#99] +Input [1]: [csales#96] Keys: [] -Functions [1]: [partial_max(csales#99)] -Aggregate Attributes [1]: [max#100] -Results [1]: [max#101] +Functions [1]: [partial_max(csales#96)] +Aggregate Attributes [1]: [max#97] +Results [1]: [max#98] (118) Exchange -Input [1]: [max#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] +Input [1]: [max#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (119) HashAggregate [codegen id : 5] -Input [1]: [max#101] +Input [1]: [max#98] Keys: [] -Functions [1]: [max(csales#99)] -Aggregate Attributes [1]: [max(csales#99)#103] -Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] +Functions [1]: [max(csales#96)] +Aggregate Attributes [1]: [max(csales#96)#100] +Results [1]: [max(csales#96)#100 AS tpcds_cmax#101] -Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (120) (120) ReusedExchange [Reuses operator id: 111] -Output [1]: [d_date_sk#90] +Output [1]: [d_date_sk#87] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] -Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] 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 7127bcfcc2b1..e69135d5ee0a 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 @@ -29,8 +29,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastExchange #4 WholeStageCodegen (4) Project [item_sk] - Filter [count(1)] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,count(1),count] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] InputAdapter Exchange [_groupingexpression,i_item_sk,d_date] #5 WholeStageCodegen (3) @@ -64,7 +64,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (9) 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))] + Filter [ssales] Subquery #3 WholeStageCodegen (5) HashAggregate [max] [max(csales),tpcds_cmax,max] @@ -102,7 +102,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] - 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,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)),ssales,sum,isEmpty] InputAdapter Exchange [c_customer_sk] #8 WholeStageCodegen (8) @@ -139,9 +139,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (14) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,sum,isEmpty] InputAdapter ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter @@ -182,9 +182,9 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] WholeStageCodegen (27) 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))] + Filter [ssales] ReusedSubquery [tpcds_cmax] #3 - 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,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)),ssales,sum,isEmpty] InputAdapter ReusedExchange [c_customer_sk,sum,isEmpty] #8 InputAdapter 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 23bfec44556c..c9132bb1e8dc 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 @@ -1,54 +1,53 @@ == Physical Plan == -* Project (50) -+- * Filter (49) - +- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (29) - : +- * SortMergeJoin Inner (28) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.item (5) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (27) - : +- Exchange (26) - : +- * Project (25) - : +- * Filter (24) - : +- * ColumnarToRow (23) - : +- Scan parquet default.store_returns (22) - +- BroadcastExchange (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildLeft (38) - :- BroadcastExchange (34) - : +- * Project (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.store (30) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet default.customer_address (35) +* Filter (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (29) + : +- * SortMergeJoin Inner (28) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * Filter (24) + : +- * ColumnarToRow (23) + : +- Scan parquet default.store_returns (22) + +- BroadcastExchange (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildLeft (38) + :- BroadcastExchange (34) + : +- * Project (33) + : +- * Filter (32) + : +- * ColumnarToRow (31) + : +- Scan parquet default.store (30) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.customer_address (35) (1) Scan parquet default.store_sales @@ -270,318 +269,314 @@ Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43 Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] Functions [1]: [sum(netpaid#39)] Aggregate Attributes [1]: [sum(netpaid#39)#45] -Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46] (49) Filter [codegen id : 14] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46, sum(netpaid#39)#47] -Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) - -(50) Project [codegen id : 14] -Output [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46, sum(netpaid#39)#47] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] +Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (104) -+- Exchange (103) - +- * HashAggregate (102) - +- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Project (98) - +- * SortMergeJoin Inner (97) - :- * Sort (91) - : +- Exchange (90) - : +- * Project (89) - : +- * SortMergeJoin Inner (88) - : :- * Sort (81) - : : +- Exchange (80) - : : +- * Project (79) - : : +- * SortMergeJoin Inner (78) - : : :- * Sort (72) - : : : +- Exchange (71) - : : : +- * Project (70) - : : : +- * SortMergeJoin Inner (69) - : : : :- * Sort (63) - : : : : +- Exchange (62) - : : : : +- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (54) - : : : : : +- * Filter (53) - : : : : : +- * ColumnarToRow (52) - : : : : : +- Scan parquet default.store_sales (51) - : : : : +- BroadcastExchange (59) - : : : : +- * Project (58) - : : : : +- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet default.store (55) - : : : +- * Sort (68) - : : : +- Exchange (67) - : : : +- * Filter (66) - : : : +- * ColumnarToRow (65) - : : : +- Scan parquet default.item (64) - : : +- * Sort (77) - : : +- Exchange (76) - : : +- * Filter (75) - : : +- * ColumnarToRow (74) - : : +- Scan parquet default.customer (73) - : +- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * Filter (84) - : +- * ColumnarToRow (83) - : +- Scan parquet default.store_returns (82) - +- * Sort (96) - +- Exchange (95) - +- * Filter (94) - +- * ColumnarToRow (93) - +- Scan parquet default.customer_address (92) - - -(51) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- * HashAggregate (100) + +- Exchange (99) + +- * HashAggregate (98) + +- * Project (97) + +- * SortMergeJoin Inner (96) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (80) + : : +- Exchange (79) + : : +- * Project (78) + : : +- * SortMergeJoin Inner (77) + : : :- * Sort (71) + : : : +- Exchange (70) + : : : +- * Project (69) + : : : +- * SortMergeJoin Inner (68) + : : : :- * Sort (62) + : : : : +- Exchange (61) + : : : : +- * Project (60) + : : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : : :- * Project (53) + : : : : : +- * Filter (52) + : : : : : +- * ColumnarToRow (51) + : : : : : +- Scan parquet default.store_sales (50) + : : : : +- BroadcastExchange (58) + : : : : +- * Project (57) + : : : : +- * Filter (56) + : : : : +- * ColumnarToRow (55) + : : : : +- Scan parquet default.store (54) + : : : +- * Sort (67) + : : : +- Exchange (66) + : : : +- * Filter (65) + : : : +- * ColumnarToRow (64) + : : : +- Scan parquet default.item (63) + : : +- * Sort (76) + : : +- Exchange (75) + : : +- * Filter (74) + : : +- * ColumnarToRow (73) + : : +- Scan parquet default.customer (72) + : +- * Sort (86) + : +- Exchange (85) + : +- * Project (84) + : +- * Filter (83) + : +- * ColumnarToRow (82) + : +- Scan parquet default.store_returns (81) + +- * Sort (95) + +- Exchange (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.customer_address (91) + + +(50) Scan parquet default.store_sales +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +(51) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -(53) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +(52) Filter [codegen id : 2] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) -(54) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +(53) Project [codegen id : 2] +Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -(55) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +(54) Scan parquet default.store +Output [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +(55) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] -(57) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +(56) Filter [codegen id : 1] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Condition : (((isnotnull(s_market_id#57) AND (s_market_id#57 = 8)) AND isnotnull(s_store_sk#55)) AND isnotnull(s_zip#59)) -(58) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +(57) Project [codegen id : 1] +Output [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] -(59) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +(58) BroadcastExchange +Input [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] -(60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +(59) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#55] Join condition: None -(61) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +(60) Project [codegen id : 2] +Output [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Input [9]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] -(62) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] +(61) Exchange +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: hashpartitioning(ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#61] -(63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 3] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: [ss_item_sk#49 ASC NULLS FIRST], false, 0 -(64) Scan parquet default.item -Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(63) Scan parquet default.item +Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(64) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -(66) Filter [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Condition : isnotnull(i_item_sk#63) +(65) Filter [codegen id : 4] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Condition : isnotnull(i_item_sk#62) -(67) Exchange -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +(66) Exchange +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(i_item_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] -(68) Sort [codegen id : 5] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 5] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [i_item_sk#62 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#63] +(68) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#62] Join condition: None -(70) Project [codegen id : 6] -Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(69) Project [codegen id : 6] +Output [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [13]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -(71) Exchange -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] +(70) Exchange +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(ss_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#69] -(72) Sort [codegen id : 7] -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 +(71) Sort [codegen id : 7] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [ss_customer_sk#50 ASC NULLS FIRST], false, 0 -(73) Scan parquet default.customer -Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(72) Scan parquet default.customer +Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(74) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(73) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -(75) Filter [codegen id : 8] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) +(74) Filter [codegen id : 8] +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(c_birth_country#73)) -(76) Exchange -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] +(75) Exchange +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(c_customer_sk#70, 5), ENSURE_REQUIREMENTS, [id=#74] -(77) Sort [codegen id : 9] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 +(76) Sort [codegen id : 9] +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 -(78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#71] +(77) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#70] Join condition: None -(79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(78) Project [codegen id : 10] +Output [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -(80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(ss_ticket_number#53, ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#76] +(79) Exchange +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#75] -(81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [ss_ticket_number#53 ASC NULLS FIRST, ss_item_sk#50 ASC NULLS FIRST], false, 0 +(80) Sort [codegen id : 11] +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 -(82) Scan parquet default.store_returns -Output [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] +(81) Scan parquet default.store_returns +Output [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 12] -Input [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] +(82) ColumnarToRow [codegen id : 12] +Input [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] -(84) Filter [codegen id : 12] -Input [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] -Condition : (isnotnull(sr_ticket_number#78) AND isnotnull(sr_item_sk#77)) +(83) Filter [codegen id : 12] +Input [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] +Condition : (isnotnull(sr_ticket_number#77) AND isnotnull(sr_item_sk#76)) -(85) Project [codegen id : 12] -Output [2]: [sr_item_sk#77, sr_ticket_number#78] -Input [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] +(84) Project [codegen id : 12] +Output [2]: [sr_item_sk#76, sr_ticket_number#77] +Input [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] -(86) Exchange -Input [2]: [sr_item_sk#77, sr_ticket_number#78] -Arguments: hashpartitioning(sr_ticket_number#78, sr_item_sk#77, 5), ENSURE_REQUIREMENTS, [id=#80] +(85) Exchange +Input [2]: [sr_item_sk#76, sr_ticket_number#77] +Arguments: hashpartitioning(sr_ticket_number#77, sr_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#79] -(87) Sort [codegen id : 13] -Input [2]: [sr_item_sk#77, sr_ticket_number#78] -Arguments: [sr_ticket_number#78 ASC NULLS FIRST, sr_item_sk#77 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +Input [2]: [sr_item_sk#76, sr_ticket_number#77] +Arguments: [sr_ticket_number#77 ASC NULLS FIRST, sr_item_sk#76 ASC NULLS FIRST], false, 0 -(88) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#53, ss_item_sk#50] -Right keys [2]: [sr_ticket_number#78, sr_item_sk#77] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] +Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join condition: None -(89) Project [codegen id : 14] -Output [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, sr_item_sk#77, sr_ticket_number#78] +(88) Project [codegen id : 14] +Output [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, sr_item_sk#76, sr_ticket_number#77] -(90) Exchange -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#81] +(89) Exchange +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(c_birth_country#73, s_zip#59, 5), ENSURE_REQUIREMENTS, [id=#80] -(91) Sort [codegen id : 15] -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +(90) Sort [codegen id : 15] +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_birth_country#73 ASC NULLS FIRST, s_zip#59 ASC NULLS FIRST], false, 0 -(92) Scan parquet default.customer_address -Output [3]: [ca_state#82, ca_zip#83, ca_country#84] +(91) Scan parquet default.customer_address +Output [3]: [ca_state#81, ca_zip#82, ca_country#83] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(93) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] -(94) Filter [codegen id : 16] -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] -Condition : (isnotnull(ca_country#84) AND isnotnull(ca_zip#83)) +(93) Filter [codegen id : 16] +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] +Condition : (isnotnull(ca_country#83) AND isnotnull(ca_zip#82)) -(95) Exchange -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] -Arguments: hashpartitioning(upper(ca_country#84), ca_zip#83, 5), ENSURE_REQUIREMENTS, [id=#85] +(94) Exchange +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] +Arguments: hashpartitioning(upper(ca_country#83), ca_zip#82, 5), ENSURE_REQUIREMENTS, [id=#84] -(96) Sort [codegen id : 17] -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] -Arguments: [upper(ca_country#84) ASC NULLS FIRST, ca_zip#83 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 17] +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] +Arguments: [upper(ca_country#83) ASC NULLS FIRST, ca_zip#82 ASC NULLS FIRST], false, 0 -(97) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#74, s_zip#60] -Right keys [2]: [upper(ca_country#84), ca_zip#83] +(96) SortMergeJoin [codegen id : 18] +Left keys [2]: [c_birth_country#73, s_zip#59] +Right keys [2]: [upper(ca_country#83), ca_zip#82] Join condition: None -(98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#82] -Input [15]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#82, ca_zip#83, ca_country#84] +(97) Project [codegen id : 18] +Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#81] +Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#81, ca_zip#82, ca_country#83] -(99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#82] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +(98) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#81] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#85] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#86] -(100) Exchange -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] -Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] +(99) Exchange +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#86] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#87] -(101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] +(100) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#86] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#88] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#88,17,2) AS netpaid#39] -(102) HashAggregate [codegen id : 19] +(101) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] -(103) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +(102) Exchange +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] -(104) HashAggregate [codegen id : 20] -Input [2]: [sum#92, count#93] +(103) HashAggregate [codegen id : 20] +Input [2]: [sum#91, count#92] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#95] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] +Aggregate Attributes [1]: [avg(netpaid#39)#94] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#94)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#95] 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 46fa83fcfed4..28524e19ed9b 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,183 +1,182 @@ WholeStageCodegen (14) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (20) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (19) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (18) - 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 [c_birth_country,s_zip,ca_country,ca_zip] - InputAdapter - WholeStageCodegen (15) - Sort [c_birth_country,s_zip] - InputAdapter - Exchange [c_birth_country,s_zip] #12 - WholeStageCodegen (14) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (11) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #13 - WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,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] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #14 - WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,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 [ss_item_sk,ss_customer_sk,ss_ticket_number,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_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (1) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #17 - WholeStageCodegen (4) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Filter [paid] + Subquery #1 + WholeStageCodegen (20) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #10 + WholeStageCodegen (19) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (18) + 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 [c_birth_country,s_zip,ca_country,ca_zip] + InputAdapter + WholeStageCodegen (15) + Sort [c_birth_country,s_zip] + InputAdapter + Exchange [c_birth_country,s_zip] #12 + WholeStageCodegen (14) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #13 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,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] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (7) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #14 + WholeStageCodegen (6) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,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 [ss_item_sk,ss_customer_sk,ss_ticket_number,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_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (1) + 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + WholeStageCodegen (5) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #17 + WholeStageCodegen (4) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #18 + WholeStageCodegen (8) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #19 + WholeStageCodegen (12) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #18 - WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - WholeStageCodegen (13) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #19 - WholeStageCodegen (12) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (17) - Sort [ca_country,ca_zip] - InputAdapter - Exchange [ca_country,ca_zip] #20 - WholeStageCodegen (16) - Filter [ca_country,ca_zip] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (17) + Sort [ca_country,ca_zip] + InputAdapter + Exchange [ca_country,ca_zip] #20 + WholeStageCodegen (16) + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + 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,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (13) + 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_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_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 [ss_store_sk,c_birth_country,s_store_sk,ca_country] + Project [ss_store_sk,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_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (7) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (6) + 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 [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] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Filter [i_color,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + WholeStageCodegen (9) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #7 + WholeStageCodegen (8) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - 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_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (13) - 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_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_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 [ss_store_sk,c_birth_country,s_store_sk,ca_country] - Project [ss_store_sk,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_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #3 - WholeStageCodegen (6) - 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 [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] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Project [s_store_sk,s_store_name,s_state,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - WholeStageCodegen (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #7 - WholeStageCodegen (8) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Project [s_store_sk,s_store_name,s_state,ca_state,ca_country] - BroadcastHashJoin [s_zip,ca_zip] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - 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_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_state,ca_zip,ca_country] + 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_state,ca_zip,ca_country] 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 527913016c99..b5c27b503791 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 @@ -1,51 +1,50 @@ == Physical Plan == -* Project (47) -+- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.store (15) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.item (22) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.customer (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * Project (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet default.store (15) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.item (22) + : +- BroadcastExchange (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet default.customer (28) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -255,273 +254,269 @@ Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42 Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] Functions [1]: [sum(netpaid#38)] Aggregate Attributes [1]: [sum(netpaid#38)#44] -Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45, sum(netpaid#38)#44 AS sum(netpaid#38)#46] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45] (46) Filter [codegen id : 11] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45, sum(netpaid#38)#46] -Condition : (isnotnull(sum(netpaid#38)#46) AND (cast(sum(netpaid#38)#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -(47) Project [codegen id : 11] -Output [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45, sum(netpaid#38)#46] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] +Condition : (isnotnull(paid#45) AND (cast(paid#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (92) -+- Exchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- Exchange (88) - +- * HashAggregate (87) - +- * Project (86) - +- * BroadcastHashJoin Inner BuildRight (85) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Project (74) - : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- * Project (68) - : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * Project (61) - : : : : +- * SortMergeJoin Inner (60) - : : : : :- * Sort (53) - : : : : : +- Exchange (52) - : : : : : +- * Project (51) - : : : : : +- * Filter (50) - : : : : : +- * ColumnarToRow (49) - : : : : : +- Scan parquet default.store_sales (48) - : : : : +- * Sort (59) - : : : : +- Exchange (58) - : : : : +- * Project (57) - : : : : +- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.store_returns (54) - : : : +- BroadcastExchange (66) - : : : +- * Project (65) - : : : +- * Filter (64) - : : : +- * ColumnarToRow (63) - : : : +- Scan parquet default.store (62) - : : +- BroadcastExchange (72) - : : +- * Filter (71) - : : +- * ColumnarToRow (70) - : : +- Scan parquet default.item (69) - : +- BroadcastExchange (78) - : +- * Filter (77) - : +- * ColumnarToRow (76) - : +- Scan parquet default.customer (75) - +- BroadcastExchange (84) - +- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.customer_address (81) - - -(48) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +* HashAggregate (91) ++- Exchange (90) + +- * HashAggregate (89) + +- * HashAggregate (88) + +- Exchange (87) + +- * HashAggregate (86) + +- * Project (85) + +- * BroadcastHashJoin Inner BuildRight (84) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * Project (73) + : : +- * BroadcastHashJoin Inner BuildRight (72) + : : :- * Project (67) + : : : +- * BroadcastHashJoin Inner BuildRight (66) + : : : :- * Project (60) + : : : : +- * SortMergeJoin Inner (59) + : : : : :- * Sort (52) + : : : : : +- Exchange (51) + : : : : : +- * Project (50) + : : : : : +- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.store_sales (47) + : : : : +- * Sort (58) + : : : : +- Exchange (57) + : : : : +- * Project (56) + : : : : +- * Filter (55) + : : : : +- * ColumnarToRow (54) + : : : : +- Scan parquet default.store_returns (53) + : : : +- BroadcastExchange (65) + : : : +- * Project (64) + : : : +- * Filter (63) + : : : +- * ColumnarToRow (62) + : : : +- Scan parquet default.store (61) + : : +- BroadcastExchange (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.item (68) + : +- BroadcastExchange (77) + : +- * Filter (76) + : +- * ColumnarToRow (75) + : +- Scan parquet default.customer (74) + +- BroadcastExchange (83) + +- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet default.customer_address (80) + + +(47) Scan parquet default.store_sales +Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +(48) ColumnarToRow [codegen id : 1] +Input [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] -(50) Filter [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +(49) Filter [codegen id : 1] +Input [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] +Condition : (((isnotnull(ss_ticket_number#51) AND isnotnull(ss_item_sk#48)) AND isnotnull(ss_store_sk#50)) AND isnotnull(ss_customer_sk#49)) -(51) Project [codegen id : 1] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +(50) Project [codegen id : 1] +Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Input [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] -(52) Exchange -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#55] +(51) Exchange +Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Arguments: hashpartitioning(ss_ticket_number#51, ss_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] -(53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 +(52) Sort [codegen id : 2] +Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Arguments: [ss_ticket_number#51 ASC NULLS FIRST, ss_item_sk#48 ASC NULLS FIRST], false, 0 -(54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +(53) Scan parquet default.store_returns +Output [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +(54) ColumnarToRow [codegen id : 3] +Input [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] -(56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] -Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) +(55) Filter [codegen id : 3] +Input [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] +Condition : (isnotnull(sr_ticket_number#56) AND isnotnull(sr_item_sk#55)) -(57) Project [codegen id : 3] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +(56) Project [codegen id : 3] +Output [2]: [sr_item_sk#55, sr_ticket_number#56] +Input [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] -(58) Exchange -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] +(57) Exchange +Input [2]: [sr_item_sk#55, sr_ticket_number#56] +Arguments: hashpartitioning(sr_ticket_number#56, sr_item_sk#55, 5), ENSURE_REQUIREMENTS, [id=#58] -(59) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 4] +Input [2]: [sr_item_sk#55, sr_ticket_number#56] +Arguments: [sr_ticket_number#56 ASC NULLS FIRST, sr_item_sk#55 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +(59) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] +Right keys [2]: [sr_ticket_number#56, sr_item_sk#55] Join condition: None -(61) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] +(60) Project [codegen id : 9] +Output [4]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52] +Input [7]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, sr_item_sk#55, sr_ticket_number#56] -(62) Scan parquet default.store -Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +(61) Scan parquet default.store +Output [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +(62) ColumnarToRow [codegen id : 5] +Input [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] -(64) Filter [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] -Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) +(63) Filter [codegen id : 5] +Input [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] +Condition : (((isnotnull(s_market_id#61) AND (s_market_id#61 = 8)) AND isnotnull(s_store_sk#59)) AND isnotnull(s_zip#63)) -(65) Project [codegen id : 5] -Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +(64) Project [codegen id : 5] +Output [4]: [s_store_sk#59, s_store_name#60, s_state#62, s_zip#63] +Input [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] -(66) BroadcastExchange -Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +(65) BroadcastExchange +Input [4]: [s_store_sk#59, s_store_name#60, s_state#62, s_zip#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#64] -(67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#60] +(66) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#50] +Right keys [1]: [s_store_sk#59] Join condition: None -(68) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +(67) Project [codegen id : 9] +Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63] +Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#59, s_store_name#60, s_state#62, s_zip#63] -(69) Scan parquet default.item -Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +(68) Scan parquet default.item +Output [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +(69) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(71) Filter [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Condition : isnotnull(i_item_sk#66) +(70) Filter [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Condition : isnotnull(i_item_sk#65) -(72) BroadcastExchange -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +(71) BroadcastExchange +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#71] -(73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#66] +(72) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#48] +Right keys [1]: [i_item_sk#65] Join condition: None -(74) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +(73) Project [codegen id : 9] +Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(75) Scan parquet default.customer -Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +(74) Scan parquet default.customer +Output [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +(75) ColumnarToRow [codegen id : 7] +Input [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) +(76) Filter [codegen id : 7] +Input [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] +Condition : (isnotnull(c_customer_sk#72) AND isnotnull(c_birth_country#75)) -(78) BroadcastExchange -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +(77) BroadcastExchange +Input [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] -(79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#73] +(78) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#49] +Right keys [1]: [c_customer_sk#72] Join condition: None -(80) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +(79) Project [codegen id : 9] +Output [12]: [ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, c_birth_country#75] +Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(81) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +(80) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +(81) ColumnarToRow [codegen id : 8] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(83) Filter [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +(82) Filter [codegen id : 8] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(84) BroadcastExchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] +(83) BroadcastExchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#80] -(85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#76, s_zip#64] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +(84) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#75, s_zip#63] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None -(86) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] +(85) Project [codegen id : 9] +Output [11]: [ss_net_paid#52, s_store_name#60, s_state#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, ca_state#77] +Input [15]: [ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, c_birth_country#75, ca_state#77, ca_zip#78, ca_country#79] -(87) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +(86) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#52, s_store_name#60, s_state#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, ca_state#77] +Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum#81] +Results [11]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#82] -(88) Exchange -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] +(87) Exchange +Input [11]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#82] +Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#83] -(89) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] +(88) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#82] +Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#84] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#84,17,2) AS netpaid#38] -(90) HashAggregate [codegen id : 10] +(89) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] -(91) Exchange -Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +(90) Exchange +Input [2]: [sum#87, count#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] -(92) HashAggregate [codegen id : 11] -Input [2]: [sum#88, count#89] +(91) HashAggregate [codegen id : 11] +Input [2]: [sum#87, count#88] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#38)#90] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#90)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#91] 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 6d50a1e3f35a..e89cd62f1c96 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,147 +1,146 @@ WholeStageCodegen (11) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (11) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (10) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (9) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #9 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + WholeStageCodegen (9) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #11 + WholeStageCodegen (1) + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #12 + WholeStageCodegen (3) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (5) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #11 - WholeStageCodegen (1) - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #12 - WholeStageCodegen (3) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (5) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (8) + Filter [ca_country,ca_zip] + ColumnarToRow InputAdapter - BroadcastExchange #14 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (8) - Filter [ca_country,ca_zip] - ColumnarToRow + 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,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (10) + 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_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 (9) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - 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_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (10) - 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_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 (9) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (1) + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #3 - WholeStageCodegen (1) - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Filter [i_color,i_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + Filter [ca_country,ca_zip] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + 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 1663ffd6dc09..0075b955f86e 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 @@ -1,54 +1,53 @@ == Physical Plan == -* Project (50) -+- * Filter (49) - +- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (29) - : +- * SortMergeJoin Inner (28) - : :- * Sort (21) - : : +- Exchange (20) - : : +- * Project (19) - : : +- * SortMergeJoin Inner (18) - : : :- * Sort (12) - : : : +- Exchange (11) - : : : +- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.item (5) - : : +- * Sort (17) - : : +- Exchange (16) - : : +- * Filter (15) - : : +- * ColumnarToRow (14) - : : +- Scan parquet default.customer (13) - : +- * Sort (27) - : +- Exchange (26) - : +- * Project (25) - : +- * Filter (24) - : +- * ColumnarToRow (23) - : +- Scan parquet default.store_returns (22) - +- BroadcastExchange (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildLeft (38) - :- BroadcastExchange (34) - : +- * Project (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.store (30) - +- * Filter (37) - +- * ColumnarToRow (36) - +- Scan parquet default.customer_address (35) +* Filter (49) ++- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (29) + : +- * SortMergeJoin Inner (28) + : :- * Sort (21) + : : +- Exchange (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * Filter (24) + : +- * ColumnarToRow (23) + : +- Scan parquet default.store_returns (22) + +- BroadcastExchange (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildLeft (38) + :- BroadcastExchange (34) + : +- * Project (33) + : +- * Filter (32) + : +- * ColumnarToRow (31) + : +- Scan parquet default.store (30) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.customer_address (35) (1) Scan parquet default.store_sales @@ -270,318 +269,314 @@ Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum#42, isEmpty#43 Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#26] Functions [1]: [sum(netpaid#39)] Aggregate Attributes [1]: [sum(netpaid#39)#45] -Results [5]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#26, sum(netpaid#39)#45 AS paid#46] (49) Filter [codegen id : 14] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46, sum(netpaid#39)#47] -Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) - -(50) Project [codegen id : 14] -Output [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] -Input [5]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46, sum(netpaid#39)#47] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#26, paid#46] +Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] -* HashAggregate (104) -+- Exchange (103) - +- * HashAggregate (102) - +- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Project (98) - +- * SortMergeJoin Inner (97) - :- * Sort (91) - : +- Exchange (90) - : +- * Project (89) - : +- * SortMergeJoin Inner (88) - : :- * Sort (81) - : : +- Exchange (80) - : : +- * Project (79) - : : +- * SortMergeJoin Inner (78) - : : :- * Sort (72) - : : : +- Exchange (71) - : : : +- * Project (70) - : : : +- * SortMergeJoin Inner (69) - : : : :- * Sort (63) - : : : : +- Exchange (62) - : : : : +- * Project (61) - : : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : : :- * Project (54) - : : : : : +- * Filter (53) - : : : : : +- * ColumnarToRow (52) - : : : : : +- Scan parquet default.store_sales (51) - : : : : +- BroadcastExchange (59) - : : : : +- * Project (58) - : : : : +- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet default.store (55) - : : : +- * Sort (68) - : : : +- Exchange (67) - : : : +- * Filter (66) - : : : +- * ColumnarToRow (65) - : : : +- Scan parquet default.item (64) - : : +- * Sort (77) - : : +- Exchange (76) - : : +- * Filter (75) - : : +- * ColumnarToRow (74) - : : +- Scan parquet default.customer (73) - : +- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * Filter (84) - : +- * ColumnarToRow (83) - : +- Scan parquet default.store_returns (82) - +- * Sort (96) - +- Exchange (95) - +- * Filter (94) - +- * ColumnarToRow (93) - +- Scan parquet default.customer_address (92) - - -(51) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#47, [id=#48] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- * HashAggregate (100) + +- Exchange (99) + +- * HashAggregate (98) + +- * Project (97) + +- * SortMergeJoin Inner (96) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (80) + : : +- Exchange (79) + : : +- * Project (78) + : : +- * SortMergeJoin Inner (77) + : : :- * Sort (71) + : : : +- Exchange (70) + : : : +- * Project (69) + : : : +- * SortMergeJoin Inner (68) + : : : :- * Sort (62) + : : : : +- Exchange (61) + : : : : +- * Project (60) + : : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : : :- * Project (53) + : : : : : +- * Filter (52) + : : : : : +- * ColumnarToRow (51) + : : : : : +- Scan parquet default.store_sales (50) + : : : : +- BroadcastExchange (58) + : : : : +- * Project (57) + : : : : +- * Filter (56) + : : : : +- * ColumnarToRow (55) + : : : : +- Scan parquet default.store (54) + : : : +- * Sort (67) + : : : +- Exchange (66) + : : : +- * Filter (65) + : : : +- * ColumnarToRow (64) + : : : +- Scan parquet default.item (63) + : : +- * Sort (76) + : : +- Exchange (75) + : : +- * Filter (74) + : : +- * ColumnarToRow (73) + : : +- Scan parquet default.customer (72) + : +- * Sort (86) + : +- Exchange (85) + : +- * Project (84) + : +- * Filter (83) + : +- * ColumnarToRow (82) + : +- Scan parquet default.store_returns (81) + +- * Sort (95) + +- Exchange (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.customer_address (91) + + +(50) Scan parquet default.store_sales +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +(51) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -(53) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +(52) Filter [codegen id : 2] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) -(54) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +(53) Project [codegen id : 2] +Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -(55) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +(54) Scan parquet default.store +Output [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +(55) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] -(57) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +(56) Filter [codegen id : 1] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] +Condition : (((isnotnull(s_market_id#57) AND (s_market_id#57 = 8)) AND isnotnull(s_store_sk#55)) AND isnotnull(s_zip#59)) -(58) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +(57) Project [codegen id : 1] +Output [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Input [5]: [s_store_sk#55, s_store_name#56, s_market_id#57, s_state#58, s_zip#59] -(59) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +(58) BroadcastExchange +Input [4]: [s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] -(60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +(59) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#55] Join condition: None -(61) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +(60) Project [codegen id : 2] +Output [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Input [9]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, s_store_sk#55, s_store_name#56, s_state#58, s_zip#59] -(62) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] +(61) Exchange +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: hashpartitioning(ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#61] -(63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +(62) Sort [codegen id : 3] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59] +Arguments: [ss_item_sk#49 ASC NULLS FIRST], false, 0 -(64) Scan parquet default.item -Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(63) Scan parquet default.item +Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(64) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -(66) Filter [codegen id : 4] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Condition : isnotnull(i_item_sk#63) +(65) Filter [codegen id : 4] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Condition : isnotnull(i_item_sk#62) -(67) Exchange -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +(66) Exchange +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(i_item_sk#62, 5), ENSURE_REQUIREMENTS, [id=#68] -(68) Sort [codegen id : 5] -Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 5] +Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [i_item_sk#62 ASC NULLS FIRST], false, 0 -(69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#63] +(68) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#62] Join condition: None -(70) Project [codegen id : 6] -Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +(69) Project [codegen id : 6] +Output [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [13]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -(71) Exchange -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] +(70) Exchange +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: hashpartitioning(ss_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#69] -(72) Sort [codegen id : 7] -Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 +(71) Sort [codegen id : 7] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Arguments: [ss_customer_sk#50 ASC NULLS FIRST], false, 0 -(73) Scan parquet default.customer -Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(72) Scan parquet default.customer +Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(74) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(73) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -(75) Filter [codegen id : 8] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) +(74) Filter [codegen id : 8] +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(c_birth_country#73)) -(76) Exchange -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] +(75) Exchange +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(c_customer_sk#70, 5), ENSURE_REQUIREMENTS, [id=#74] -(77) Sort [codegen id : 9] -Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 +(76) Sort [codegen id : 9] +Input [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 -(78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#71] +(77) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#70] Join condition: None -(79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +(78) Project [codegen id : 10] +Output [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_customer_sk#50, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -(80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(ss_ticket_number#53, ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#76] +(79) Exchange +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#75] -(81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [ss_ticket_number#53 ASC NULLS FIRST, ss_item_sk#50 ASC NULLS FIRST], false, 0 +(80) Sort [codegen id : 11] +Input [14]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 -(82) Scan parquet default.store_returns -Output [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] +(81) Scan parquet default.store_returns +Output [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(83) ColumnarToRow [codegen id : 12] -Input [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] +(82) ColumnarToRow [codegen id : 12] +Input [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] -(84) Filter [codegen id : 12] -Input [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] -Condition : (isnotnull(sr_ticket_number#78) AND isnotnull(sr_item_sk#77)) +(83) Filter [codegen id : 12] +Input [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] +Condition : (isnotnull(sr_ticket_number#77) AND isnotnull(sr_item_sk#76)) -(85) Project [codegen id : 12] -Output [2]: [sr_item_sk#77, sr_ticket_number#78] -Input [3]: [sr_item_sk#77, sr_ticket_number#78, sr_returned_date_sk#79] +(84) Project [codegen id : 12] +Output [2]: [sr_item_sk#76, sr_ticket_number#77] +Input [3]: [sr_item_sk#76, sr_ticket_number#77, sr_returned_date_sk#78] -(86) Exchange -Input [2]: [sr_item_sk#77, sr_ticket_number#78] -Arguments: hashpartitioning(sr_ticket_number#78, sr_item_sk#77, 5), ENSURE_REQUIREMENTS, [id=#80] +(85) Exchange +Input [2]: [sr_item_sk#76, sr_ticket_number#77] +Arguments: hashpartitioning(sr_ticket_number#77, sr_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#79] -(87) Sort [codegen id : 13] -Input [2]: [sr_item_sk#77, sr_ticket_number#78] -Arguments: [sr_ticket_number#78 ASC NULLS FIRST, sr_item_sk#77 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +Input [2]: [sr_item_sk#76, sr_ticket_number#77] +Arguments: [sr_ticket_number#77 ASC NULLS FIRST, sr_item_sk#76 ASC NULLS FIRST], false, 0 -(88) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#53, ss_item_sk#50] -Right keys [2]: [sr_ticket_number#78, sr_item_sk#77] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] +Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join condition: None -(89) Project [codegen id : 14] -Output [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Input [16]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, sr_item_sk#77, sr_ticket_number#78] +(88) Project [codegen id : 14] +Output [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Input [16]: [ss_item_sk#49, ss_ticket_number#52, ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, sr_item_sk#76, sr_ticket_number#77] -(90) Exchange -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#81] +(89) Exchange +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: hashpartitioning(c_birth_country#73, s_zip#59, 5), ENSURE_REQUIREMENTS, [id=#80] -(91) Sort [codegen id : 15] -Input [12]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] -Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +(90) Sort [codegen id : 15] +Input [12]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73] +Arguments: [c_birth_country#73 ASC NULLS FIRST, s_zip#59 ASC NULLS FIRST], false, 0 -(92) Scan parquet default.customer_address -Output [3]: [ca_state#82, ca_zip#83, ca_country#84] +(91) Scan parquet default.customer_address +Output [3]: [ca_state#81, ca_zip#82, ca_country#83] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(93) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] -(94) Filter [codegen id : 16] -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] -Condition : (isnotnull(ca_country#84) AND isnotnull(ca_zip#83)) +(93) Filter [codegen id : 16] +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] +Condition : (isnotnull(ca_country#83) AND isnotnull(ca_zip#82)) -(95) Exchange -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] -Arguments: hashpartitioning(upper(ca_country#84), ca_zip#83, 5), ENSURE_REQUIREMENTS, [id=#85] +(94) Exchange +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] +Arguments: hashpartitioning(upper(ca_country#83), ca_zip#82, 5), ENSURE_REQUIREMENTS, [id=#84] -(96) Sort [codegen id : 17] -Input [3]: [ca_state#82, ca_zip#83, ca_country#84] -Arguments: [upper(ca_country#84) ASC NULLS FIRST, ca_zip#83 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 17] +Input [3]: [ca_state#81, ca_zip#82, ca_country#83] +Arguments: [upper(ca_country#83) ASC NULLS FIRST, ca_zip#82 ASC NULLS FIRST], false, 0 -(97) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#74, s_zip#60] -Right keys [2]: [upper(ca_country#84), ca_zip#83] +(96) SortMergeJoin [codegen id : 18] +Left keys [2]: [c_birth_country#73, s_zip#59] +Right keys [2]: [upper(ca_country#83), ca_zip#82] Join condition: None -(98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#82] -Input [15]: [ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#82, ca_zip#83, ca_country#84] +(97) Project [codegen id : 18] +Output [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#81] +Input [15]: [ss_net_paid#53, s_store_name#56, s_state#58, s_zip#59, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, c_birth_country#73, ca_state#81, ca_zip#82, ca_country#83] -(99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#82] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +(98) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#53, s_store_name#56, s_state#58, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_first_name#71, c_last_name#72, ca_state#81] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#85] +Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#86] -(100) Exchange -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] -Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] +(99) Exchange +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#86] +Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#87] -(101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] -Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#82, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] +(100) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#86] +Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#81, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#88] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#88,17,2) AS netpaid#39] -(102) HashAggregate [codegen id : 19] +(101) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] -(103) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +(102) Exchange +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] -(104) HashAggregate [codegen id : 20] -Input [2]: [sum#92, count#93] +(103) HashAggregate [codegen id : 20] +Input [2]: [sum#91, count#92] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#95] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] +Aggregate Attributes [1]: [avg(netpaid#39)#94] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#94)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#95] 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 46fa83fcfed4..28524e19ed9b 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,183 +1,182 @@ WholeStageCodegen (14) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (20) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (19) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (18) - 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 [c_birth_country,s_zip,ca_country,ca_zip] - InputAdapter - WholeStageCodegen (15) - Sort [c_birth_country,s_zip] - InputAdapter - Exchange [c_birth_country,s_zip] #12 - WholeStageCodegen (14) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (11) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #13 - WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,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] - SortMergeJoin [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #14 - WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,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 [ss_item_sk,ss_customer_sk,ss_ticket_number,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_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (1) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #17 - WholeStageCodegen (4) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Filter [paid] + Subquery #1 + WholeStageCodegen (20) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #10 + WholeStageCodegen (19) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (18) + 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 [c_birth_country,s_zip,ca_country,ca_zip] + InputAdapter + WholeStageCodegen (15) + Sort [c_birth_country,s_zip] + InputAdapter + Exchange [c_birth_country,s_zip] #12 + WholeStageCodegen (14) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #13 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,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] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (7) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #14 + WholeStageCodegen (6) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,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 [ss_item_sk,ss_customer_sk,ss_ticket_number,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_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (1) + 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + WholeStageCodegen (5) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #17 + WholeStageCodegen (4) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #18 + WholeStageCodegen (8) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #19 + WholeStageCodegen (12) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #18 - WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - WholeStageCodegen (13) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #19 - WholeStageCodegen (12) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - WholeStageCodegen (17) - Sort [ca_country,ca_zip] - InputAdapter - Exchange [ca_country,ca_zip] #20 - WholeStageCodegen (16) - Filter [ca_country,ca_zip] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (17) + Sort [ca_country,ca_zip] + InputAdapter + Exchange [ca_country,ca_zip] #20 + WholeStageCodegen (16) + Filter [ca_country,ca_zip] + ColumnarToRow + InputAdapter + 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,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (13) + 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_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_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 [ss_store_sk,c_birth_country,s_store_sk,ca_country] + Project [ss_store_sk,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_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (7) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (6) + 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 [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] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + Filter [i_color,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + WholeStageCodegen (5) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #6 + WholeStageCodegen (4) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + WholeStageCodegen (9) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #7 + WholeStageCodegen (8) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] ColumnarToRow InputAdapter - 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_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (13) - 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_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_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 [ss_store_sk,c_birth_country,s_store_sk,ca_country] - Project [ss_store_sk,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_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (7) - Sort [ss_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #3 - WholeStageCodegen (6) - 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 [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] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Project [s_store_sk,s_store_name,s_state,ca_state,ca_country] + BroadcastHashJoin [s_zip,ca_zip] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - WholeStageCodegen (5) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (4) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - WholeStageCodegen (9) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #7 - WholeStageCodegen (8) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Project [s_store_sk,s_store_name,s_state,ca_state,ca_country] - BroadcastHashJoin [s_zip,ca_zip] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - 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_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_state,ca_zip,ca_country] + 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_state,ca_zip,ca_country] 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 2b6c25252d1d..0fe97bb0a228 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 @@ -1,51 +1,50 @@ == Physical Plan == -* Project (47) -+- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.store (15) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.item (22) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.customer (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * Project (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet default.store (15) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.item (22) + : +- BroadcastExchange (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet default.customer (28) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -255,273 +254,269 @@ Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum#41, isEmpty#42 Keys [3]: [c_last_name#27, c_first_name#26, s_store_name#13] Functions [1]: [sum(netpaid#38)] Aggregate Attributes [1]: [sum(netpaid#38)#44] -Results [5]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45, sum(netpaid#38)#44 AS sum(netpaid#38)#46] +Results [4]: [c_last_name#27, c_first_name#26, s_store_name#13, sum(netpaid#38)#44 AS paid#45] (46) Filter [codegen id : 11] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45, sum(netpaid#38)#46] -Condition : (isnotnull(sum(netpaid#38)#46) AND (cast(sum(netpaid#38)#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -(47) Project [codegen id : 11] -Output [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] -Input [5]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45, sum(netpaid#38)#46] +Input [4]: [c_last_name#27, c_first_name#26, s_store_name#13, paid#45] +Condition : (isnotnull(paid#45) AND (cast(paid#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (92) -+- Exchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- Exchange (88) - +- * HashAggregate (87) - +- * Project (86) - +- * BroadcastHashJoin Inner BuildRight (85) - :- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Project (74) - : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- * Project (68) - : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * Project (61) - : : : : +- * SortMergeJoin Inner (60) - : : : : :- * Sort (53) - : : : : : +- Exchange (52) - : : : : : +- * Project (51) - : : : : : +- * Filter (50) - : : : : : +- * ColumnarToRow (49) - : : : : : +- Scan parquet default.store_sales (48) - : : : : +- * Sort (59) - : : : : +- Exchange (58) - : : : : +- * Project (57) - : : : : +- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet default.store_returns (54) - : : : +- BroadcastExchange (66) - : : : +- * Project (65) - : : : +- * Filter (64) - : : : +- * ColumnarToRow (63) - : : : +- Scan parquet default.store (62) - : : +- BroadcastExchange (72) - : : +- * Filter (71) - : : +- * ColumnarToRow (70) - : : +- Scan parquet default.item (69) - : +- BroadcastExchange (78) - : +- * Filter (77) - : +- * ColumnarToRow (76) - : +- Scan parquet default.customer (75) - +- BroadcastExchange (84) - +- * Filter (83) - +- * ColumnarToRow (82) - +- Scan parquet default.customer_address (81) - - -(48) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +* HashAggregate (91) ++- Exchange (90) + +- * HashAggregate (89) + +- * HashAggregate (88) + +- Exchange (87) + +- * HashAggregate (86) + +- * Project (85) + +- * BroadcastHashJoin Inner BuildRight (84) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * Project (73) + : : +- * BroadcastHashJoin Inner BuildRight (72) + : : :- * Project (67) + : : : +- * BroadcastHashJoin Inner BuildRight (66) + : : : :- * Project (60) + : : : : +- * SortMergeJoin Inner (59) + : : : : :- * Sort (52) + : : : : : +- Exchange (51) + : : : : : +- * Project (50) + : : : : : +- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.store_sales (47) + : : : : +- * Sort (58) + : : : : +- Exchange (57) + : : : : +- * Project (56) + : : : : +- * Filter (55) + : : : : +- * ColumnarToRow (54) + : : : : +- Scan parquet default.store_returns (53) + : : : +- BroadcastExchange (65) + : : : +- * Project (64) + : : : +- * Filter (63) + : : : +- * ColumnarToRow (62) + : : : +- Scan parquet default.store (61) + : : +- BroadcastExchange (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.item (68) + : +- BroadcastExchange (77) + : +- * Filter (76) + : +- * ColumnarToRow (75) + : +- Scan parquet default.customer (74) + +- BroadcastExchange (83) + +- * Filter (82) + +- * ColumnarToRow (81) + +- Scan parquet default.customer_address (80) + + +(47) Scan parquet default.store_sales +Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +(48) ColumnarToRow [codegen id : 1] +Input [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] -(50) Filter [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +(49) Filter [codegen id : 1] +Input [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] +Condition : (((isnotnull(ss_ticket_number#51) AND isnotnull(ss_item_sk#48)) AND isnotnull(ss_store_sk#50)) AND isnotnull(ss_customer_sk#49)) -(51) Project [codegen id : 1] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +(50) Project [codegen id : 1] +Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Input [6]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, ss_sold_date_sk#53] -(52) Exchange -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: hashpartitioning(ss_ticket_number#52, ss_item_sk#49, 5), ENSURE_REQUIREMENTS, [id=#55] +(51) Exchange +Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Arguments: hashpartitioning(ss_ticket_number#51, ss_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] -(53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST], false, 0 +(52) Sort [codegen id : 2] +Input [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Arguments: [ss_ticket_number#51 ASC NULLS FIRST, ss_item_sk#48 ASC NULLS FIRST], false, 0 -(54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +(53) Scan parquet default.store_returns +Output [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +(54) ColumnarToRow [codegen id : 3] +Input [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] -(56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] -Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) +(55) Filter [codegen id : 3] +Input [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] +Condition : (isnotnull(sr_ticket_number#56) AND isnotnull(sr_item_sk#55)) -(57) Project [codegen id : 3] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +(56) Project [codegen id : 3] +Output [2]: [sr_item_sk#55, sr_ticket_number#56] +Input [3]: [sr_item_sk#55, sr_ticket_number#56, sr_returned_date_sk#57] -(58) Exchange -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] +(57) Exchange +Input [2]: [sr_item_sk#55, sr_ticket_number#56] +Arguments: hashpartitioning(sr_ticket_number#56, sr_item_sk#55, 5), ENSURE_REQUIREMENTS, [id=#58] -(59) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 4] +Input [2]: [sr_item_sk#55, sr_ticket_number#56] +Arguments: [sr_ticket_number#56 ASC NULLS FIRST, sr_item_sk#55 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +(59) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] +Right keys [2]: [sr_ticket_number#56, sr_item_sk#55] Join condition: None -(61) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] +(60) Project [codegen id : 9] +Output [4]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52] +Input [7]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52, sr_item_sk#55, sr_ticket_number#56] -(62) Scan parquet default.store -Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +(61) Scan parquet default.store +Output [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +(62) ColumnarToRow [codegen id : 5] +Input [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] -(64) Filter [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] -Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) +(63) Filter [codegen id : 5] +Input [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] +Condition : (((isnotnull(s_market_id#61) AND (s_market_id#61 = 8)) AND isnotnull(s_store_sk#59)) AND isnotnull(s_zip#63)) -(65) Project [codegen id : 5] -Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +(64) Project [codegen id : 5] +Output [4]: [s_store_sk#59, s_store_name#60, s_state#62, s_zip#63] +Input [5]: [s_store_sk#59, s_store_name#60, s_market_id#61, s_state#62, s_zip#63] -(66) BroadcastExchange -Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +(65) BroadcastExchange +Input [4]: [s_store_sk#59, s_store_name#60, s_state#62, s_zip#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#64] -(67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#60] +(66) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#50] +Right keys [1]: [s_store_sk#59] Join condition: None -(68) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +(67) Project [codegen id : 9] +Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63] +Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#59, s_store_name#60, s_state#62, s_zip#63] -(69) Scan parquet default.item -Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +(68) Scan parquet default.item +Output [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +(69) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(71) Filter [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Condition : isnotnull(i_item_sk#66) +(70) Filter [codegen id : 6] +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Condition : isnotnull(i_item_sk#65) -(72) BroadcastExchange -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +(71) BroadcastExchange +Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#71] -(73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#66] +(72) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#48] +Right keys [1]: [i_item_sk#65] Join condition: None -(74) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +(73) Project [codegen id : 9] +Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -(75) Scan parquet default.customer -Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +(74) Scan parquet default.customer +Output [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +(75) ColumnarToRow [codegen id : 7] +Input [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) +(76) Filter [codegen id : 7] +Input [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] +Condition : (isnotnull(c_customer_sk#72) AND isnotnull(c_birth_country#75)) -(78) BroadcastExchange -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +(77) BroadcastExchange +Input [4]: [c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] -(79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#73] +(78) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#49] +Right keys [1]: [c_customer_sk#72] Join condition: None -(80) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +(79) Project [codegen id : 9] +Output [12]: [ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, c_birth_country#75] +Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_customer_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(81) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +(80) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +(81) ColumnarToRow [codegen id : 8] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(83) Filter [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +(82) Filter [codegen id : 8] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(84) BroadcastExchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] +(83) BroadcastExchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#80] -(85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#76, s_zip#64] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +(84) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#75, s_zip#63] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None -(86) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] +(85) Project [codegen id : 9] +Output [11]: [ss_net_paid#52, s_store_name#60, s_state#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, ca_state#77] +Input [15]: [ss_net_paid#52, s_store_name#60, s_state#62, s_zip#63, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, c_birth_country#75, ca_state#77, ca_zip#78, ca_country#79] -(87) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +(86) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#52, s_store_name#60, s_state#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#73, c_last_name#74, ca_state#77] +Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum#81] +Results [11]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#82] -(88) Exchange -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] +(87) Exchange +Input [11]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#82] +Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#83] -(89) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] +(88) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#82] +Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#60, ca_state#77, s_state#62, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] +Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#84] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#84,17,2) AS netpaid#38] -(90) HashAggregate [codegen id : 10] +(89) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] -(91) Exchange -Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +(90) Exchange +Input [2]: [sum#87, count#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] -(92) HashAggregate [codegen id : 11] -Input [2]: [sum#88, count#89] +(91) HashAggregate [codegen id : 11] +Input [2]: [sum#87, count#88] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#38)#90] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#90)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#91] 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 6d50a1e3f35a..e89cd62f1c96 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,147 +1,146 @@ WholeStageCodegen (11) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (11) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - WholeStageCodegen (10) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (9) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #9 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + WholeStageCodegen (9) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #11 + WholeStageCodegen (1) + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #12 + WholeStageCodegen (3) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (5) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #11 - WholeStageCodegen (1) - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #12 - WholeStageCodegen (3) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (5) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (8) + Filter [ca_country,ca_zip] + ColumnarToRow InputAdapter - BroadcastExchange #14 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (8) - Filter [ca_country,ca_zip] - ColumnarToRow + 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,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (10) + 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_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 (9) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - 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_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (10) - 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_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 (9) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (1) + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #3 - WholeStageCodegen (1) - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Filter [i_color,i_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #4 - WholeStageCodegen (3) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + Filter [c_customer_sk,c_birth_country] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + Filter [ca_country,ca_zip] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [c_customer_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Filter [ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_state,ca_zip,ca_country] + 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/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 9ed490449b00..ba9207bdc69f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -306,11 +306,11 @@ Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, c Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] Aggregate Attributes [10]: [count(1)#62, min(cd_dep_count#31)#63, max(cd_dep_count#31)#64, avg(cd_dep_count#31)#65, min(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, avg(cd_dep_employed_count#32)#68, min(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, avg(cd_dep_college_count#33)#71] -Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31 AS aggOrder#84] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31] (57) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, aggOrder#84] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, aggOrder#84 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, cd_dep_count#31] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] ===== Subqueries ===== 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 820be2b5fd58..04d73d66e487 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,6 +1,6 @@ -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)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,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 (22) - 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(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(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),aggOrder,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + 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(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(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),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (21) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 48ae82483445..4798532f2ae7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -251,11 +251,11 @@ Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, c Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [10]: [count(1), min(cd_dep_count#28), max(cd_dep_count#28), avg(cd_dep_count#28), min(cd_dep_employed_count#29), max(cd_dep_employed_count#29), avg(cd_dep_employed_count#29), min(cd_dep_college_count#30), max(cd_dep_college_count#30), avg(cd_dep_college_count#30)] Aggregate Attributes [10]: [count(1)#59, min(cd_dep_count#28)#60, max(cd_dep_count#28)#61, avg(cd_dep_count#28)#62, min(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, avg(cd_dep_employed_count#29)#65, min(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, avg(cd_dep_college_count#30)#68] -Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28 AS aggOrder#81] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28] (46) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, aggOrder#81] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, aggOrder#81 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, cd_dep_count#28] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] ===== Subqueries ===== 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 2614f4f8ae88..da69e04e2ea6 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,6 +1,6 @@ -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)] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,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_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(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(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),aggOrder,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + 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(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(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),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index d75c9039fd21..8fa5abffaa52 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -1,36 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (14) - : : : +- * Filter (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * Filter (8) - : : : +- * HashAggregate (7) - : : : +- Exchange (6) - : : : +- * HashAggregate (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (20) - : : +- * Project (19) - : : +- * Filter (18) - : : +- Window (17) - : : +- * Sort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.item (23) - +- ReusedExchange (29) +TakeOrderedAndProject (31) ++- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * Project (13) + : : : +- * Filter (12) + : : : +- Window (11) + : : : +- * Sort (10) + : : : +- Exchange (9) + : : : +- * Filter (8) + : : : +- * HashAggregate (7) + : : : +- Exchange (6) + : : : +- * HashAggregate (5) + : : : +- * Project (4) + : : : +- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.store_sales (1) + : : +- BroadcastExchange (19) + : : +- * Project (18) + : : +- * Filter (17) + : : +- Window (16) + : : +- * Sort (15) + : : +- ReusedExchange (14) + : +- BroadcastExchange (25) + : +- * Filter (24) + : +- * ColumnarToRow (23) + : +- Scan parquet default.item (22) + +- ReusedExchange (28) (1) Scan parquet default.store_sales @@ -67,157 +66,153 @@ Input [3]: [ss_item_sk#1, sum#7, count#8] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#10] -Results [3]: [ss_item_sk#1 AS item_sk#11, cast((avg(UnscaledValue(ss_net_profit#3))#10 / 100.0) as decimal(11,6)) AS rank_col#12, cast((avg(UnscaledValue(ss_net_profit#3))#10 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#13] +Results [2]: [ss_item_sk#1 AS item_sk#11, cast((avg(UnscaledValue(ss_net_profit#3))#10 / 100.0) as decimal(11,6)) AS rank_col#12] (8) Filter [codegen id : 2] -Input [3]: [item_sk#11, rank_col#12, avg(ss_net_profit#3)#13] -Condition : (isnotnull(avg(ss_net_profit#3)#13) AND (cast(avg(ss_net_profit#3)#13 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) - -(9) Project [codegen id : 2] -Output [2]: [item_sk#11, rank_col#12] -Input [3]: [item_sk#11, rank_col#12, avg(ss_net_profit#3)#13] +Input [2]: [item_sk#11, rank_col#12] +Condition : (isnotnull(rank_col#12) AND (cast(rank_col#12 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#13, [id=#14])), DecimalType(13,7), true))) -(10) Exchange +(9) Exchange Input [2]: [item_sk#11, rank_col#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#15] -(11) Sort [codegen id : 3] +(10) Sort [codegen id : 3] Input [2]: [item_sk#11, rank_col#12] Arguments: [rank_col#12 ASC NULLS FIRST], false, 0 -(12) Window +(11) Window Input [2]: [item_sk#11, rank_col#12] -Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#17], [rank_col#12 ASC NULLS FIRST] +Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#16], [rank_col#12 ASC NULLS FIRST] -(13) Filter [codegen id : 10] -Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +(12) Filter [codegen id : 10] +Input [3]: [item_sk#11, rank_col#12, rnk#16] +Condition : ((rnk#16 < 11) AND isnotnull(item_sk#11)) -(14) Project [codegen id : 10] -Output [2]: [item_sk#11, rnk#17] -Input [3]: [item_sk#11, rank_col#12, rnk#17] +(13) Project [codegen id : 10] +Output [2]: [item_sk#11, rnk#16] +Input [3]: [item_sk#11, rank_col#12, rnk#16] -(15) ReusedExchange [Reuses operator id: 10] -Output [2]: [item_sk#18, rank_col#19] +(14) ReusedExchange [Reuses operator id: 9] +Output [2]: [item_sk#17, rank_col#18] -(16) Sort [codegen id : 6] -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank_col#19 DESC NULLS LAST], false, 0 +(15) Sort [codegen id : 6] +Input [2]: [item_sk#17, rank_col#18] +Arguments: [rank_col#18 DESC NULLS LAST], false, 0 -(17) Window -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST] +(16) Window +Input [2]: [item_sk#17, rank_col#18] +Arguments: [rank(rank_col#18) windowspecdefinition(rank_col#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#19], [rank_col#18 DESC NULLS LAST] -(18) Filter [codegen id : 7] -Input [3]: [item_sk#18, rank_col#19, rnk#20] -Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18)) +(17) Filter [codegen id : 7] +Input [3]: [item_sk#17, rank_col#18, rnk#19] +Condition : ((rnk#19 < 11) AND isnotnull(item_sk#17)) -(19) Project [codegen id : 7] -Output [2]: [item_sk#18, rnk#20] -Input [3]: [item_sk#18, rank_col#19, rnk#20] +(18) Project [codegen id : 7] +Output [2]: [item_sk#17, rnk#19] +Input [3]: [item_sk#17, rank_col#18, rnk#19] -(20) BroadcastExchange -Input [2]: [item_sk#18, rnk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#21] +(19) BroadcastExchange +Input [2]: [item_sk#17, rnk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#20] -(21) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [rnk#17] -Right keys [1]: [rnk#20] +(20) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [rnk#16] +Right keys [1]: [rnk#19] Join condition: None -(22) Project [codegen id : 10] -Output [3]: [item_sk#11, rnk#17, item_sk#18] -Input [4]: [item_sk#11, rnk#17, item_sk#18, rnk#20] +(21) Project [codegen id : 10] +Output [3]: [item_sk#11, rnk#16, item_sk#17] +Input [4]: [item_sk#11, rnk#16, item_sk#17, rnk#19] -(23) Scan parquet default.item -Output [2]: [i_item_sk#22, i_product_name#23] +(22) Scan parquet default.item +Output [2]: [i_item_sk#21, i_product_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#22, i_product_name#23] +(23) ColumnarToRow [codegen id : 8] +Input [2]: [i_item_sk#21, i_product_name#22] -(25) Filter [codegen id : 8] -Input [2]: [i_item_sk#22, i_product_name#23] -Condition : isnotnull(i_item_sk#22) +(24) Filter [codegen id : 8] +Input [2]: [i_item_sk#21, i_product_name#22] +Condition : isnotnull(i_item_sk#21) -(26) BroadcastExchange -Input [2]: [i_item_sk#22, i_product_name#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] +(25) BroadcastExchange +Input [2]: [i_item_sk#21, i_product_name#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(27) BroadcastHashJoin [codegen id : 10] +(26) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#21] Join condition: None -(28) Project [codegen id : 10] -Output [3]: [rnk#17, item_sk#18, i_product_name#23] -Input [5]: [item_sk#11, rnk#17, item_sk#18, i_item_sk#22, i_product_name#23] +(27) Project [codegen id : 10] +Output [3]: [rnk#16, item_sk#17, i_product_name#22] +Input [5]: [item_sk#11, rnk#16, item_sk#17, i_item_sk#21, i_product_name#22] -(29) ReusedExchange [Reuses operator id: 26] -Output [2]: [i_item_sk#25, i_product_name#26] +(28) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#24, i_product_name#25] -(30) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#18] -Right keys [1]: [i_item_sk#25] +(29) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [item_sk#17] +Right keys [1]: [i_item_sk#24] Join condition: None -(31) Project [codegen id : 10] -Output [3]: [rnk#17, i_product_name#23 AS best_performing#27, i_product_name#26 AS worst_performing#28] -Input [5]: [rnk#17, item_sk#18, i_product_name#23, i_item_sk#25, i_product_name#26] +(30) Project [codegen id : 10] +Output [3]: [rnk#16, i_product_name#22 AS best_performing#26, i_product_name#25 AS worst_performing#27] +Input [5]: [rnk#16, item_sk#17, i_product_name#22, i_item_sk#24, i_product_name#25] -(32) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#27, worst_performing#28] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#27, worst_performing#28] +(31) TakeOrderedAndProject +Input [3]: [rnk#16, best_performing#26, worst_performing#27] +Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#26, worst_performing#27] ===== Subqueries ===== -Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* HashAggregate (39) -+- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * Filter (35) - +- * ColumnarToRow (34) - +- Scan parquet default.store_sales (33) +Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#13, [id=#14] +* HashAggregate (38) ++- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * Filter (34) + +- * ColumnarToRow (33) + +- Scan parquet default.store_sales (32) -(33) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] +(32) Scan parquet default.store_sales +Output [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] - -(35) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] -Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29)) - -(36) Project [codegen id : 1] -Output [2]: [ss_store_sk#30, ss_net_profit#31] -Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] - -(37) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#30, ss_net_profit#31] -Keys [1]: [ss_store_sk#30] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))] -Aggregate Attributes [2]: [sum#33, count#34] -Results [3]: [ss_store_sk#30, sum#35, count#36] - -(38) Exchange -Input [3]: [ss_store_sk#30, sum#35, count#36] -Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [id=#37] - -(39) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#30, sum#35, count#36] -Keys [1]: [ss_store_sk#30] -Functions [1]: [avg(UnscaledValue(ss_net_profit#31))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#38] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#38 / 100.0) as decimal(11,6)) AS rank_col#39] +(33) ColumnarToRow [codegen id : 1] +Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] + +(34) Filter [codegen id : 1] +Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_store_sk#29) AND (ss_store_sk#29 = 4)) AND isnull(ss_addr_sk#28)) + +(35) Project [codegen id : 1] +Output [2]: [ss_store_sk#29, ss_net_profit#30] +Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] + +(36) HashAggregate [codegen id : 1] +Input [2]: [ss_store_sk#29, ss_net_profit#30] +Keys [1]: [ss_store_sk#29] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#30))] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#29, sum#34, count#35] + +(37) Exchange +Input [3]: [ss_store_sk#29, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#29, 5), ENSURE_REQUIREMENTS, [id=#36] + +(38) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#29, sum#34, count#35] +Keys [1]: [ss_store_sk#29] +Functions [1]: [avg(UnscaledValue(ss_net_profit#30))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#30))#37] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#30))#37 / 100.0) as decimal(11,6)) AS rank_col#38] 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 ffc04edf3dfb..cfff698cda36 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 @@ -15,30 +15,29 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter Exchange #1 WholeStageCodegen (2) - Project [item_sk,rank_col] - Filter [avg(ss_net_profit)] - Subquery #1 - WholeStageCodegen (2) - 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_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_store_sk,ss_net_profit,ss_sold_date_sk] - 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] [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_store_sk,ss_net_profit,ss_sold_date_sk] + Filter [rank_col] + Subquery #1 + WholeStageCodegen (2) + 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_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_store_sk,ss_net_profit,ss_sold_date_sk] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (1) + 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_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter BroadcastExchange #4 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 6bb069211533..b3d0081f5d22 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,37 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (15) - : : : +- * Project (14) - : : : +- * Filter (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * Filter (8) - : : : +- * HashAggregate (7) - : : : +- Exchange (6) - : : : +- * HashAggregate (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * Sort (17) - : : +- ReusedExchange (16) - : +- BroadcastExchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.item (24) - +- ReusedExchange (30) +TakeOrderedAndProject (32) ++- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) + : :- * Project (22) + : : +- * SortMergeJoin Inner (21) + : : :- * Sort (14) + : : : +- * Project (13) + : : : +- * Filter (12) + : : : +- Window (11) + : : : +- * Sort (10) + : : : +- Exchange (9) + : : : +- * Filter (8) + : : : +- * HashAggregate (7) + : : : +- Exchange (6) + : : : +- * HashAggregate (5) + : : : +- * Project (4) + : : : +- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.store_sales (1) + : : +- * Sort (20) + : : +- * Project (19) + : : +- * Filter (18) + : : +- Window (17) + : : +- * Sort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.item (23) + +- ReusedExchange (29) (1) Scan parquet default.store_sales @@ -68,161 +67,157 @@ Input [3]: [ss_item_sk#1, sum#7, count#8] Keys [1]: [ss_item_sk#1] Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#10] -Results [3]: [ss_item_sk#1 AS item_sk#11, cast((avg(UnscaledValue(ss_net_profit#3))#10 / 100.0) as decimal(11,6)) AS rank_col#12, cast((avg(UnscaledValue(ss_net_profit#3))#10 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#13] +Results [2]: [ss_item_sk#1 AS item_sk#11, cast((avg(UnscaledValue(ss_net_profit#3))#10 / 100.0) as decimal(11,6)) AS rank_col#12] (8) Filter [codegen id : 2] -Input [3]: [item_sk#11, rank_col#12, avg(ss_net_profit#3)#13] -Condition : (isnotnull(avg(ss_net_profit#3)#13) AND (cast(avg(ss_net_profit#3)#13 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) - -(9) Project [codegen id : 2] -Output [2]: [item_sk#11, rank_col#12] -Input [3]: [item_sk#11, rank_col#12, avg(ss_net_profit#3)#13] +Input [2]: [item_sk#11, rank_col#12] +Condition : (isnotnull(rank_col#12) AND (cast(rank_col#12 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#13, [id=#14])), DecimalType(13,7), true))) -(10) Exchange +(9) Exchange Input [2]: [item_sk#11, rank_col#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#16] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#15] -(11) Sort [codegen id : 3] +(10) Sort [codegen id : 3] Input [2]: [item_sk#11, rank_col#12] Arguments: [rank_col#12 ASC NULLS FIRST], false, 0 -(12) Window +(11) Window Input [2]: [item_sk#11, rank_col#12] -Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#17], [rank_col#12 ASC NULLS FIRST] +Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#16], [rank_col#12 ASC NULLS FIRST] -(13) Filter [codegen id : 4] -Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +(12) Filter [codegen id : 4] +Input [3]: [item_sk#11, rank_col#12, rnk#16] +Condition : ((rnk#16 < 11) AND isnotnull(item_sk#11)) -(14) Project [codegen id : 4] -Output [2]: [item_sk#11, rnk#17] -Input [3]: [item_sk#11, rank_col#12, rnk#17] +(13) Project [codegen id : 4] +Output [2]: [item_sk#11, rnk#16] +Input [3]: [item_sk#11, rank_col#12, rnk#16] -(15) Sort [codegen id : 4] -Input [2]: [item_sk#11, rnk#17] -Arguments: [rnk#17 ASC NULLS FIRST], false, 0 +(14) Sort [codegen id : 4] +Input [2]: [item_sk#11, rnk#16] +Arguments: [rnk#16 ASC NULLS FIRST], false, 0 -(16) ReusedExchange [Reuses operator id: 10] -Output [2]: [item_sk#18, rank_col#19] +(15) ReusedExchange [Reuses operator id: 9] +Output [2]: [item_sk#17, rank_col#18] -(17) Sort [codegen id : 7] -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank_col#19 DESC NULLS LAST], false, 0 +(16) Sort [codegen id : 7] +Input [2]: [item_sk#17, rank_col#18] +Arguments: [rank_col#18 DESC NULLS LAST], false, 0 -(18) Window -Input [2]: [item_sk#18, rank_col#19] -Arguments: [rank(rank_col#19) windowspecdefinition(rank_col#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#20], [rank_col#19 DESC NULLS LAST] +(17) Window +Input [2]: [item_sk#17, rank_col#18] +Arguments: [rank(rank_col#18) windowspecdefinition(rank_col#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#19], [rank_col#18 DESC NULLS LAST] -(19) Filter [codegen id : 8] -Input [3]: [item_sk#18, rank_col#19, rnk#20] -Condition : ((rnk#20 < 11) AND isnotnull(item_sk#18)) +(18) Filter [codegen id : 8] +Input [3]: [item_sk#17, rank_col#18, rnk#19] +Condition : ((rnk#19 < 11) AND isnotnull(item_sk#17)) -(20) Project [codegen id : 8] -Output [2]: [item_sk#18, rnk#20] -Input [3]: [item_sk#18, rank_col#19, rnk#20] +(19) Project [codegen id : 8] +Output [2]: [item_sk#17, rnk#19] +Input [3]: [item_sk#17, rank_col#18, rnk#19] -(21) Sort [codegen id : 8] -Input [2]: [item_sk#18, rnk#20] -Arguments: [rnk#20 ASC NULLS FIRST], false, 0 +(20) Sort [codegen id : 8] +Input [2]: [item_sk#17, rnk#19] +Arguments: [rnk#19 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 11] -Left keys [1]: [rnk#17] -Right keys [1]: [rnk#20] +(21) SortMergeJoin [codegen id : 11] +Left keys [1]: [rnk#16] +Right keys [1]: [rnk#19] Join condition: None -(23) Project [codegen id : 11] -Output [3]: [item_sk#11, rnk#17, item_sk#18] -Input [4]: [item_sk#11, rnk#17, item_sk#18, rnk#20] +(22) Project [codegen id : 11] +Output [3]: [item_sk#11, rnk#16, item_sk#17] +Input [4]: [item_sk#11, rnk#16, item_sk#17, rnk#19] -(24) Scan parquet default.item -Output [2]: [i_item_sk#21, i_product_name#22] +(23) Scan parquet default.item +Output [2]: [i_item_sk#20, i_product_name#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#21, i_product_name#22] +(24) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_sk#20, i_product_name#21] -(26) Filter [codegen id : 9] -Input [2]: [i_item_sk#21, i_product_name#22] -Condition : isnotnull(i_item_sk#21) +(25) Filter [codegen id : 9] +Input [2]: [i_item_sk#20, i_product_name#21] +Condition : isnotnull(i_item_sk#20) -(27) BroadcastExchange -Input [2]: [i_item_sk#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +(26) BroadcastExchange +Input [2]: [i_item_sk#20, i_product_name#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(28) BroadcastHashJoin [codegen id : 11] +(27) BroadcastHashJoin [codegen id : 11] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#21] +Right keys [1]: [i_item_sk#20] Join condition: None -(29) Project [codegen id : 11] -Output [3]: [rnk#17, item_sk#18, i_product_name#22] -Input [5]: [item_sk#11, rnk#17, item_sk#18, i_item_sk#21, i_product_name#22] +(28) Project [codegen id : 11] +Output [3]: [rnk#16, item_sk#17, i_product_name#21] +Input [5]: [item_sk#11, rnk#16, item_sk#17, i_item_sk#20, i_product_name#21] -(30) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#24, i_product_name#25] +(29) ReusedExchange [Reuses operator id: 26] +Output [2]: [i_item_sk#23, i_product_name#24] -(31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#18] -Right keys [1]: [i_item_sk#24] +(30) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [item_sk#17] +Right keys [1]: [i_item_sk#23] Join condition: None -(32) Project [codegen id : 11] -Output [3]: [rnk#17, i_product_name#22 AS best_performing#26, i_product_name#25 AS worst_performing#27] -Input [5]: [rnk#17, item_sk#18, i_product_name#22, i_item_sk#24, i_product_name#25] +(31) Project [codegen id : 11] +Output [3]: [rnk#16, i_product_name#21 AS best_performing#25, i_product_name#24 AS worst_performing#26] +Input [5]: [rnk#16, item_sk#17, i_product_name#21, i_item_sk#23, i_product_name#24] -(33) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#26, worst_performing#27] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#26, worst_performing#27] +(32) TakeOrderedAndProject +Input [3]: [rnk#16, best_performing#25, worst_performing#26] +Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#25, worst_performing#26] ===== Subqueries ===== -Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_sales (34) +Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#13, [id=#14] +* HashAggregate (39) ++- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * Filter (35) + +- * ColumnarToRow (34) + +- Scan parquet default.store_sales (33) -(34) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] +(33) Scan parquet default.store_sales +Output [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] - -(36) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] -Condition : ((isnotnull(ss_store_sk#29) AND (ss_store_sk#29 = 4)) AND isnull(ss_addr_sk#28)) - -(37) Project [codegen id : 1] -Output [2]: [ss_store_sk#29, ss_net_profit#30] -Input [4]: [ss_addr_sk#28, ss_store_sk#29, ss_net_profit#30, ss_sold_date_sk#31] - -(38) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#29, ss_net_profit#30] -Keys [1]: [ss_store_sk#29] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#30))] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#29, sum#34, count#35] - -(39) Exchange -Input [3]: [ss_store_sk#29, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#29, 5), ENSURE_REQUIREMENTS, [id=#36] - -(40) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#29, sum#34, count#35] -Keys [1]: [ss_store_sk#29] -Functions [1]: [avg(UnscaledValue(ss_net_profit#30))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#30))#37] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#30))#37 / 100.0) as decimal(11,6)) AS rank_col#38] +(34) ColumnarToRow [codegen id : 1] +Input [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] + +(35) Filter [codegen id : 1] +Input [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] +Condition : ((isnotnull(ss_store_sk#28) AND (ss_store_sk#28 = 4)) AND isnull(ss_addr_sk#27)) + +(36) Project [codegen id : 1] +Output [2]: [ss_store_sk#28, ss_net_profit#29] +Input [4]: [ss_addr_sk#27, ss_store_sk#28, ss_net_profit#29, ss_sold_date_sk#30] + +(37) HashAggregate [codegen id : 1] +Input [2]: [ss_store_sk#28, ss_net_profit#29] +Keys [1]: [ss_store_sk#28] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#29))] +Aggregate Attributes [2]: [sum#31, count#32] +Results [3]: [ss_store_sk#28, sum#33, count#34] + +(38) Exchange +Input [3]: [ss_store_sk#28, sum#33, count#34] +Arguments: hashpartitioning(ss_store_sk#28, 5), ENSURE_REQUIREMENTS, [id=#35] + +(39) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#28, sum#33, count#34] +Keys [1]: [ss_store_sk#28] +Functions [1]: [avg(UnscaledValue(ss_net_profit#29))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#29))#36] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#29))#36 / 100.0) as decimal(11,6)) AS rank_col#37] 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 f3642f6d949b..ab9b90f2fe4d 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 @@ -18,30 +18,29 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter Exchange #1 WholeStageCodegen (2) - Project [item_sk,rank_col] - Filter [avg(ss_net_profit)] - Subquery #1 - WholeStageCodegen (2) - 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_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_store_sk,ss_net_profit,ss_sold_date_sk] - 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] [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_store_sk,ss_net_profit,ss_sold_date_sk] + Filter [rank_col] + Subquery #1 + WholeStageCodegen (2) + 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_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_store_sk,ss_net_profit,ss_sold_date_sk] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (1) + 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_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter WholeStageCodegen (8) Sort [rnk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index a84042dafb0f..ebb85479a89d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -221,15 +221,15 @@ Input [2]: [ca_zip#16, count#22] Keys [1]: [ca_zip#16] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#24] -Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS count(1)#26] +Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS cnt#26] (38) Filter [codegen id : 10] -Input [2]: [ca_zip#25, count(1)#26] -Condition : (count(1)#26 > 10) +Input [2]: [ca_zip#25, cnt#26] +Condition : (cnt#26 > 10) (39) Project [codegen id : 10] Output [1]: [ca_zip#25] -Input [2]: [ca_zip#25, count(1)#26] +Input [2]: [ca_zip#25, cnt#26] (40) BroadcastExchange Input [1]: [ca_zip#25] 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 88cc9c98e1ed..485cdcbea04e 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 @@ -59,8 +59,8 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] BroadcastExchange #7 WholeStageCodegen (10) Project [ca_zip] - Filter [count(1)] - HashAggregate [ca_zip,count] [count(1),ca_zip,count(1),count] + Filter [cnt] + HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] InputAdapter Exchange [ca_zip] #8 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index d19397c262a8..e4a9e5529b84 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -195,15 +195,15 @@ Input [2]: [ca_zip#15, count#20] Keys [1]: [ca_zip#15] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] -Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS count(1)#24] +Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS cnt#24] (33) Filter [codegen id : 5] -Input [2]: [ca_zip#23, count(1)#24] -Condition : (count(1)#24 > 10) +Input [2]: [ca_zip#23, cnt#24] +Condition : (cnt#24 > 10) (34) Project [codegen id : 5] Output [1]: [ca_zip#23] -Input [2]: [ca_zip#23, count(1)#24] +Input [2]: [ca_zip#23, cnt#24] (35) BroadcastExchange Input [1]: [ca_zip#23] 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 294468c04eea..da328c123e10 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 @@ -50,8 +50,8 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] BroadcastExchange #6 WholeStageCodegen (5) Project [ca_zip] - Filter [count(1)] - HashAggregate [ca_zip,count] [count(1),ca_zip,count(1),count] + Filter [cnt] + HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] InputAdapter Exchange [ca_zip] #7 WholeStageCodegen (4) 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 e1ce480288a4..e2140bf17476 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 @@ -1,120 +1,118 @@ == Physical Plan == -TakeOrderedAndProject (116) -+- * BroadcastHashJoin Inner BuildRight (115) - :- * Project (90) - : +- * Filter (89) - : +- * HashAggregate (88) - : +- Exchange (87) - : +- * HashAggregate (86) - : +- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * Project (74) - : : +- * BroadcastHashJoin Inner BuildRight (73) - : : :- * SortMergeJoin LeftSemi (67) - : : : :- * Sort (5) - : : : : +- Exchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- * Sort (66) - : : : +- Exchange (65) - : : : +- * Project (64) - : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : :- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.item (6) - : : : +- BroadcastExchange (62) - : : : +- * HashAggregate (61) - : : : +- Exchange (60) - : : : +- * HashAggregate (59) - : : : +- * SortMergeJoin LeftSemi (58) - : : : :- * Sort (46) - : : : : +- Exchange (45) - : : : : +- * HashAggregate (44) - : : : : +- Exchange (43) - : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- * SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) - : : : +- * Sort (57) - : : : +- Exchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Project (52) - : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : :- * Filter (49) - : : : : : +- * ColumnarToRow (48) - : : : : : +- Scan parquet default.web_sales (47) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (72) - : : +- * Project (71) - : : +- * Filter (70) - : : +- * ColumnarToRow (69) - : : +- Scan parquet default.date_dim (68) - : +- BroadcastExchange (83) - : +- * SortMergeJoin LeftSemi (82) - : :- * Sort (79) - : : +- Exchange (78) - : : +- * Filter (77) - : : +- * ColumnarToRow (76) - : : +- Scan parquet default.item (75) - : +- * Sort (81) - : +- ReusedExchange (80) - +- BroadcastExchange (114) - +- * Project (113) - +- * Filter (112) - +- * HashAggregate (111) - +- Exchange (110) - +- * HashAggregate (109) - +- * Project (108) - +- * BroadcastHashJoin Inner BuildRight (107) - :- * Project (105) - : +- * BroadcastHashJoin Inner BuildRight (104) - : :- * SortMergeJoin LeftSemi (98) - : : :- * Sort (95) - : : : +- Exchange (94) - : : : +- * Filter (93) - : : : +- * ColumnarToRow (92) - : : : +- Scan parquet default.store_sales (91) - : : +- * Sort (97) - : : +- ReusedExchange (96) - : +- BroadcastExchange (103) - : +- * Project (102) - : +- * Filter (101) - : +- * ColumnarToRow (100) - : +- Scan parquet default.date_dim (99) - +- ReusedExchange (106) +TakeOrderedAndProject (114) ++- * BroadcastHashJoin Inner BuildRight (113) + :- * Filter (89) + : +- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * Project (85) + : +- * BroadcastHashJoin Inner BuildRight (84) + : :- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- * SortMergeJoin LeftSemi (67) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- * Sort (66) + : : : +- Exchange (65) + : : : +- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.item (6) + : : : +- BroadcastExchange (62) + : : : +- * HashAggregate (61) + : : : +- Exchange (60) + : : : +- * HashAggregate (59) + : : : +- * SortMergeJoin LeftSemi (58) + : : : :- * Sort (46) + : : : : +- Exchange (45) + : : : : +- * HashAggregate (44) + : : : : +- Exchange (43) + : : : : +- * HashAggregate (42) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- * SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) + : : : +- * Sort (57) + : : : +- Exchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : :- * Filter (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- Scan parquet default.web_sales (47) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (72) + : : +- * Project (71) + : : +- * Filter (70) + : : +- * ColumnarToRow (69) + : : +- Scan parquet default.date_dim (68) + : +- BroadcastExchange (83) + : +- * SortMergeJoin LeftSemi (82) + : :- * Sort (79) + : : +- Exchange (78) + : : +- * Filter (77) + : : +- * ColumnarToRow (76) + : : +- Scan parquet default.item (75) + : +- * Sort (81) + : +- ReusedExchange (80) + +- BroadcastExchange (112) + +- * Filter (111) + +- * HashAggregate (110) + +- Exchange (109) + +- * HashAggregate (108) + +- * Project (107) + +- * BroadcastHashJoin Inner BuildRight (106) + :- * Project (104) + : +- * BroadcastHashJoin Inner BuildRight (103) + : :- * SortMergeJoin LeftSemi (97) + : : :- * Sort (94) + : : : +- Exchange (93) + : : : +- * Filter (92) + : : : +- * ColumnarToRow (91) + : : : +- Scan parquet default.store_sales (90) + : : +- * Sort (96) + : : +- ReusedExchange (95) + : +- BroadcastExchange (102) + : +- * Project (101) + : +- * Filter (100) + : +- * ColumnarToRow (99) + : +- Scan parquet default.date_dim (98) + +- ReusedExchange (105) (1) Scan parquet default.store_sales @@ -518,289 +516,281 @@ Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Results [6]: [store AS channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#70, count(1)#68 AS number_sales#71] (89) Filter [codegen id : 92] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +Input [6]: [channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sales#70, number_sales#71] +Condition : (isnotnull(sales#70) AND (cast(sales#70 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 92] -Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] - -(91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +(90) Scan parquet default.store_sales +Output [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +(91) ColumnarToRow [codegen id : 46] +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -(93) Filter [codegen id : 46] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_item_sk#75) +(92) Filter [codegen id : 46] +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Condition : isnotnull(ss_item_sk#74) -(94) Exchange -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] +(93) Exchange +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Arguments: hashpartitioning(ss_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#79] -(95) Sort [codegen id : 47] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 +(94) Sort [codegen id : 47] +Input [4]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Arguments: [ss_item_sk#74 ASC NULLS FIRST], false, 0 -(96) ReusedExchange [Reuses operator id: unknown] +(95) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 66] +(96) Sort [codegen id : 66] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#75] +(97) SortMergeJoin [codegen id : 90] +Left keys [1]: [ss_item_sk#74] Right keys [1]: [ss_item_sk#47] Join condition: None -(99) Scan parquet default.date_dim -Output [2]: [d_date_sk#81, d_week_seq#82] +(98) Scan parquet default.date_dim +Output [2]: [d_date_sk#80, d_week_seq#81] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 67] -Input [2]: [d_date_sk#81, d_week_seq#82] +(99) ColumnarToRow [codegen id : 67] +Input [2]: [d_date_sk#80, d_week_seq#81] -(101) Filter [codegen id : 67] -Input [2]: [d_date_sk#81, d_week_seq#82] -Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) +(100) Filter [codegen id : 67] +Input [2]: [d_date_sk#80, d_week_seq#81] +Condition : ((isnotnull(d_week_seq#81) AND (d_week_seq#81 = Subquery scalar-subquery#82, [id=#83])) AND isnotnull(d_date_sk#80)) -(102) Project [codegen id : 67] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_week_seq#82] +(101) Project [codegen id : 67] +Output [1]: [d_date_sk#80] +Input [2]: [d_date_sk#80, d_week_seq#81] -(103) BroadcastExchange -Input [1]: [d_date_sk#81] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] +(102) BroadcastExchange +Input [1]: [d_date_sk#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] -(104) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#81] +(103) BroadcastHashJoin [codegen id : 90] +Left keys [1]: [ss_sold_date_sk#77] +Right keys [1]: [d_date_sk#80] Join condition: None -(105) Project [codegen id : 90] -Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] -Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] +(104) Project [codegen id : 90] +Output [3]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76] +Input [5]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#80] -(106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +(105) ReusedExchange [Reuses operator id: 83] +Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(107) BroadcastHashJoin [codegen id : 90] -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [i_item_sk#86] +(106) BroadcastHashJoin [codegen id : 90] +Left keys [1]: [ss_item_sk#74] +Right keys [1]: [i_item_sk#85] Join condition: None -(108) Project [codegen id : 90] -Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] - -(109) HashAggregate [codegen id : 90] -Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] - -(110) Exchange -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] - -(111) HashAggregate [codegen id : 91] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] - -(112) Filter [codegen id : 91] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) - -(113) Project [codegen id : 91] -Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] - -(114) BroadcastExchange -Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] - -(115) BroadcastHashJoin [codegen id : 92] +(107) Project [codegen id : 90] +Output [5]: [ss_quantity#75, ss_list_price#76, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [7]: [ss_item_sk#74, ss_quantity#75, ss_list_price#76, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] + +(108) HashAggregate [codegen id : 90] +Input [5]: [ss_quantity#75, ss_list_price#76, i_brand_id#86, i_class_id#87, i_category_id#88] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#92, isEmpty#93, count#94] + +(109) Exchange +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] + +(110) HashAggregate [codegen id : 91] +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [store AS channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#99, count(1)#97 AS number_sales#100] + +(111) Filter [codegen id : 91] +Input [6]: [channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Condition : (isnotnull(sales#99) AND (cast(sales#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) + +(112) BroadcastExchange +Input [6]: [channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#101] + +(113) BroadcastHashJoin [codegen id : 92] Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] -Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Right keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] Join condition: None -(116) TakeOrderedAndProject -Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +(114) TakeOrderedAndProject +Input [12]: [channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sales#70, number_sales#71, channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#69, i_brand_id#55, i_class_id#56, i_category_id#57, sales#70, number_sales#71, channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] -* HashAggregate (139) -+- Exchange (138) - +- * HashAggregate (137) - +- Union (136) - :- * Project (125) - : +- * BroadcastHashJoin Inner BuildRight (124) - : :- * ColumnarToRow (118) - : : +- Scan parquet default.store_sales (117) - : +- BroadcastExchange (123) - : +- * Project (122) - : +- * Filter (121) - : +- * ColumnarToRow (120) - : +- Scan parquet default.date_dim (119) - :- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * ColumnarToRow (127) - : : +- Scan parquet default.catalog_sales (126) - : +- ReusedExchange (128) - +- * Project (135) - +- * BroadcastHashJoin Inner BuildRight (134) - :- * ColumnarToRow (132) - : +- Scan parquet default.web_sales (131) - +- ReusedExchange (133) - - -(117) Scan parquet default.store_sales -Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +* HashAggregate (137) ++- Exchange (136) + +- * HashAggregate (135) + +- Union (134) + :- * Project (123) + : +- * BroadcastHashJoin Inner BuildRight (122) + : :- * ColumnarToRow (116) + : : +- Scan parquet default.store_sales (115) + : +- BroadcastExchange (121) + : +- * Project (120) + : +- * Filter (119) + : +- * ColumnarToRow (118) + : +- Scan parquet default.date_dim (117) + :- * Project (128) + : +- * BroadcastHashJoin Inner BuildRight (127) + : :- * ColumnarToRow (125) + : : +- Scan parquet default.catalog_sales (124) + : +- ReusedExchange (126) + +- * Project (133) + +- * BroadcastHashJoin Inner BuildRight (132) + :- * ColumnarToRow (130) + : +- Scan parquet default.web_sales (129) + +- ReusedExchange (131) + + +(115) Scan parquet default.store_sales +Output [3]: [ss_quantity#102, ss_list_price#103, ss_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ss_sold_date_sk#104), dynamicpruningexpression(ss_sold_date_sk#104 IN dynamicpruning#105)] ReadSchema: struct -(118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +(116) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#102, ss_list_price#103, ss_sold_date_sk#104] -(119) Scan parquet default.date_dim -Output [2]: [d_date_sk#108, d_year#109] +(117) Scan parquet default.date_dim +Output [2]: [d_date_sk#106, d_year#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] +(118) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#106, d_year#107] -(121) Filter [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] -Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1998)) AND (d_year#109 <= 2000)) AND isnotnull(d_date_sk#108)) +(119) Filter [codegen id : 1] +Input [2]: [d_date_sk#106, d_year#107] +Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1998)) AND (d_year#107 <= 2000)) AND isnotnull(d_date_sk#106)) -(122) Project [codegen id : 1] -Output [1]: [d_date_sk#108] -Input [2]: [d_date_sk#108, d_year#109] +(120) Project [codegen id : 1] +Output [1]: [d_date_sk#106] +Input [2]: [d_date_sk#106, d_year#107] -(123) BroadcastExchange -Input [1]: [d_date_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] +(121) BroadcastExchange +Input [1]: [d_date_sk#106] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] -(124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#106] -Right keys [1]: [d_date_sk#108] +(122) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#104] +Right keys [1]: [d_date_sk#106] Join condition: None -(125) Project [codegen id : 2] -Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] -Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] +(123) Project [codegen id : 2] +Output [2]: [ss_quantity#102 AS quantity#109, ss_list_price#103 AS list_price#110] +Input [4]: [ss_quantity#102, ss_list_price#103, ss_sold_date_sk#104, d_date_sk#106] -(126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +(124) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#111, cs_list_price#112, cs_sold_date_sk#113] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(cs_sold_date_sk#113), dynamicpruningexpression(cs_sold_date_sk#113 IN dynamicpruning#105)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +(125) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#111, cs_list_price#112, cs_sold_date_sk#113] -(128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#116] +(126) ReusedExchange [Reuses operator id: 121] +Output [1]: [d_date_sk#114] -(129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +(127) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#113] +Right keys [1]: [d_date_sk#114] Join condition: None -(130) Project [codegen id : 4] -Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] -Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] +(128) Project [codegen id : 4] +Output [2]: [cs_quantity#111 AS quantity#115, cs_list_price#112 AS list_price#116] +Input [4]: [cs_quantity#111, cs_list_price#112, cs_sold_date_sk#113, d_date_sk#114] -(131) Scan parquet default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +(129) Scan parquet default.web_sales +Output [3]: [ws_quantity#117, ws_list_price#118, ws_sold_date_sk#119] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#119), dynamicpruningexpression(ws_sold_date_sk#119 IN dynamicpruning#105)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +(130) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#117, ws_list_price#118, ws_sold_date_sk#119] -(133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#122] +(131) ReusedExchange [Reuses operator id: 121] +Output [1]: [d_date_sk#120] -(134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +(132) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#119] +Right keys [1]: [d_date_sk#120] Join condition: None -(135) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] +(133) Project [codegen id : 6] +Output [2]: [ws_quantity#117 AS quantity#121, ws_list_price#118 AS list_price#122] +Input [4]: [ws_quantity#117, ws_list_price#118, ws_sold_date_sk#119, d_date_sk#120] -(136) Union +(134) Union -(137) HashAggregate [codegen id : 7] -Input [2]: [quantity#111, list_price#112] +(135) HashAggregate [codegen id : 7] +Input [2]: [quantity#109, list_price#110] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#125, count#126] -Results [2]: [sum#127, count#128] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#123, count#124] +Results [2]: [sum#125, count#126] -(138) Exchange -Input [2]: [sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +(136) Exchange +Input [2]: [sum#125, count#126] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#127] -(139) HashAggregate [codegen id : 8] -Input [2]: [sum#127, count#128] +(137) HashAggregate [codegen id : 8] +Input [2]: [sum#125, count#126] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))#128] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#109 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#110 as decimal(12,2)))), DecimalType(18,2), true))#128 AS average_sales#129] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 -ReusedExchange (140) +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 +ReusedExchange (138) -(140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +(138) ReusedExchange [Reuses operator id: 121] +Output [1]: [d_date_sk#106] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 +Subquery:3 Hosting operator id = 124 Hosting Expression = cs_sold_date_sk#113 IN dynamicpruning#105 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 +Subquery:4 Hosting operator id = 129 Hosting Expression = ws_sold_date_sk#119 IN dynamicpruning#105 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (141) +ReusedExchange (139) -(141) ReusedExchange [Reuses operator id: 72] +(139) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#49] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -ReusedExchange (142) +ReusedExchange (140) -(142) ReusedExchange [Reuses operator id: 16] +(140) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#14] Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 @@ -808,62 +798,62 @@ Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN d Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* Project (146) -+- * Filter (145) - +- * ColumnarToRow (144) - +- Scan parquet default.date_dim (143) +* Project (144) ++- * Filter (143) + +- * ColumnarToRow (142) + +- Scan parquet default.date_dim (141) -(143) Scan parquet default.date_dim -Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +(141) Scan parquet default.date_dim +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +(142) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -(145) Filter [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 1999)) AND (d_moy#134 = 12)) AND (d_dom#135 = 16)) +(143) Filter [codegen id : 1] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) -(146) Project [codegen id : 1] -Output [1]: [d_week_seq#132] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +(144) Project [codegen id : 1] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:10 Hosting operator id = 111 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 -ReusedExchange (147) +Subquery:11 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#78 +ReusedExchange (145) -(147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#81] +(145) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#80] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] -* Project (151) -+- * Filter (150) - +- * ColumnarToRow (149) - +- Scan parquet default.date_dim (148) +Subquery:12 Hosting operator id = 100 Hosting Expression = Subquery scalar-subquery#82, [id=#83] +* Project (149) ++- * Filter (148) + +- * ColumnarToRow (147) + +- Scan parquet default.date_dim (146) -(148) Scan parquet default.date_dim -Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +(146) Scan parquet default.date_dim +Output [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +(147) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] -(150) Filter [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] -Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1998)) AND (d_moy#138 = 12)) AND (d_dom#139 = 16)) +(148) Filter [codegen id : 1] +Input [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] +Condition : (((((isnotnull(d_year#135) AND isnotnull(d_moy#136)) AND isnotnull(d_dom#137)) AND (d_year#135 = 1998)) AND (d_moy#136 = 12)) AND (d_dom#137 = 16)) -(151) Project [codegen id : 1] -Output [1]: [d_week_seq#136] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +(149) Project [codegen id : 1] +Output [1]: [d_week_seq#134] +Input [4]: [d_week_seq#134, d_year#135, d_moy#136, d_dom#137] 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 6a8fcada7a61..f1d25b6e5130 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,261 +1,259 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (92) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [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 #4 - WholeStageCodegen (8) - 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 #18 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - ReusedExchange [d_date_sk] #19 + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + 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 #18 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + ReusedExchange [d_date_sk] #19 + InputAdapter + BroadcastExchange #19 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #19 + Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #19 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #19 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (45) + 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] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #2 + WholeStageCodegen (1) + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + InputAdapter + WholeStageCodegen (21) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (20) + Project [i_item_sk] + 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_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (19) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (22) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #3 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #19 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #19 - 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),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_class_id,i_category_id] #1 - WholeStageCodegen (45) - 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] - SortMergeJoin [ss_item_sk,ss_item_sk] + 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 #15 + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] + WholeStageCodegen (24) + Sort [i_item_sk] InputAdapter - Exchange [ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk] + Exchange [i_item_sk] #16 + WholeStageCodegen (23) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (21) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (20) - Project [i_item_sk] - 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] + ReusedExchange [ss_item_sk] #17 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (91) + Filter [sales] + ReusedSubquery [average_sales] #4 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #21 + WholeStageCodegen (90) + 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] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (47) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #22 + WholeStageCodegen (46) + Filter [ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (19) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - 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] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (22) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ColumnarToRow + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #6 + ReusedExchange [d_date_sk] #23 + InputAdapter + WholeStageCodegen (66) + Sort [ss_item_sk] InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (44) - SortMergeJoin [i_item_sk,ss_item_sk] - InputAdapter - WholeStageCodegen (24) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #16 - WholeStageCodegen (23) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #17 InputAdapter - WholeStageCodegen (43) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #17 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (91) - Project [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] #4 - 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),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_class_id,i_category_id] #21 - WholeStageCodegen (90) - 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] - SortMergeJoin [ss_item_sk,ss_item_sk] - InputAdapter - WholeStageCodegen (47) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #22 - WholeStageCodegen (46) - Filter [ss_item_sk] + BroadcastExchange #23 + WholeStageCodegen (67) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #7 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #6 - ReusedExchange [d_date_sk] #23 - InputAdapter - WholeStageCodegen (66) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #17 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (67) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #7 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - 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_item_sk,i_brand_id,i_class_id,i_category_id] #15 + 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_item_sk,i_brand_id,i_class_id,i_category_id] #15 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 bdafb17d69d4..9fbf0347fee2 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 @@ -1,104 +1,102 @@ == Physical Plan == -TakeOrderedAndProject (100) -+- * BroadcastHashJoin Inner BuildRight (99) - :- * Project (77) - : +- * Filter (76) - : +- * HashAggregate (75) - : +- Exchange (74) - : +- * HashAggregate (73) - : +- * Project (72) - : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (56) - : : : +- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.item (4) - : : : +- BroadcastExchange (53) - : : : +- * HashAggregate (52) - : : : +- * HashAggregate (51) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : : : :- * HashAggregate (39) - : : : : +- Exchange (38) - : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) - : : : +- BroadcastExchange (49) - : : : +- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Filter (42) - : : : : : +- * ColumnarToRow (41) - : : : : : +- Scan parquet default.web_sales (40) - : : : : +- ReusedExchange (43) - : : : +- ReusedExchange (46) - : : +- BroadcastExchange (63) - : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : :- * Filter (60) - : : : +- * ColumnarToRow (59) - : : : +- Scan parquet default.item (58) - : : +- ReusedExchange (61) - : +- BroadcastExchange (70) - : +- * Project (69) - : +- * Filter (68) - : +- * ColumnarToRow (67) - : +- Scan parquet default.date_dim (66) - +- BroadcastExchange (98) - +- * Project (97) - +- * Filter (96) - +- * HashAggregate (95) - +- Exchange (94) - +- * HashAggregate (93) - +- * Project (92) - +- * BroadcastHashJoin Inner BuildRight (91) - :- * Project (85) - : +- * BroadcastHashJoin Inner BuildRight (84) - : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : :- * Filter (80) - : : : +- * ColumnarToRow (79) - : : : +- Scan parquet default.store_sales (78) - : : +- ReusedExchange (81) - : +- ReusedExchange (83) - +- BroadcastExchange (90) - +- * Project (89) - +- * Filter (88) - +- * ColumnarToRow (87) - +- Scan parquet default.date_dim (86) +TakeOrderedAndProject (98) ++- * BroadcastHashJoin Inner BuildRight (97) + :- * Filter (76) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * HashAggregate (73) + : +- * Project (72) + : +- * BroadcastHashJoin Inner BuildRight (71) + : :- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * BroadcastHashJoin LeftSemi BuildRight (57) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (56) + : : : +- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Filter (6) + : : : : +- * ColumnarToRow (5) + : : : : +- Scan parquet default.item (4) + : : : +- BroadcastExchange (53) + : : : +- * HashAggregate (52) + : : : +- * HashAggregate (51) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : : : :- * HashAggregate (39) + : : : : +- Exchange (38) + : : : : +- * HashAggregate (37) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) + : : : +- BroadcastExchange (49) + : : : +- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * Project (45) + : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : :- * Filter (42) + : : : : : +- * ColumnarToRow (41) + : : : : : +- Scan parquet default.web_sales (40) + : : : : +- ReusedExchange (43) + : : : +- ReusedExchange (46) + : : +- BroadcastExchange (63) + : : +- * BroadcastHashJoin LeftSemi BuildRight (62) + : : :- * Filter (60) + : : : +- * ColumnarToRow (59) + : : : +- Scan parquet default.item (58) + : : +- ReusedExchange (61) + : +- BroadcastExchange (70) + : +- * Project (69) + : +- * Filter (68) + : +- * ColumnarToRow (67) + : +- Scan parquet default.date_dim (66) + +- BroadcastExchange (96) + +- * Filter (95) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * Project (91) + +- * BroadcastHashJoin Inner BuildRight (90) + :- * Project (84) + : +- * BroadcastHashJoin Inner BuildRight (83) + : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : :- * Filter (79) + : : : +- * ColumnarToRow (78) + : : : +- Scan parquet default.store_sales (77) + : : +- ReusedExchange (80) + : +- ReusedExchange (82) + +- BroadcastExchange (89) + +- * Project (88) + +- * Filter (87) + +- * ColumnarToRow (86) + +- Scan parquet default.date_dim (85) (1) Scan parquet default.store_sales @@ -450,277 +448,269 @@ Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Results [6]: [store AS channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#65, count(1)#63 AS number_sales#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sales#65, number_sales#66] +Condition : (isnotnull(sales#65) AND (cast(sales#65 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) -(77) Project [codegen id : 52] -Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] - -(78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +(77) Scan parquet default.store_sales +Output [4]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#72), dynamicpruningexpression(ss_sold_date_sk#72 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +(78) ColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72] -(80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_item_sk#70) +(79) Filter [codegen id : 50] +Input [4]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72] +Condition : isnotnull(ss_item_sk#69) -(81) ReusedExchange [Reuses operator id: unknown] +(80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#43] -(82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] +(81) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#69] Right keys [1]: [ss_item_sk#43] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +(82) ReusedExchange [Reuses operator id: 63] +Output [4]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] -(84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [i_item_sk#75] +(83) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#69] +Right keys [1]: [i_item_sk#74] Join condition: None -(85) Project [codegen id : 50] -Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +(84) Project [codegen id : 50] +Output [6]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77] +Input [8]: [ss_item_sk#69, ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77] -(86) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_week_seq#80] +(85) Scan parquet default.date_dim +Output [2]: [d_date_sk#78, d_week_seq#79] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] +(86) ColumnarToRow [codegen id : 49] +Input [2]: [d_date_sk#78, d_week_seq#79] -(88) Filter [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] -Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) +(87) Filter [codegen id : 49] +Input [2]: [d_date_sk#78, d_week_seq#79] +Condition : ((isnotnull(d_week_seq#79) AND (d_week_seq#79 = Subquery scalar-subquery#80, [id=#81])) AND isnotnull(d_date_sk#78)) -(89) Project [codegen id : 49] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_week_seq#80] +(88) Project [codegen id : 49] +Output [1]: [d_date_sk#78] +Input [2]: [d_date_sk#78, d_week_seq#79] -(90) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +(89) BroadcastExchange +Input [1]: [d_date_sk#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#82] -(91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#79] +(90) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#72] +Right keys [1]: [d_date_sk#78] Join condition: None -(92) Project [codegen id : 50] -Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] - -(93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] - -(94) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] - -(95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] - -(96) Filter [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) - -(97) Project [codegen id : 51] -Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] - -(98) BroadcastExchange -Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] - -(99) BroadcastHashJoin [codegen id : 52] +(91) Project [codegen id : 50] +Output [5]: [ss_quantity#70, ss_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77] +Input [7]: [ss_quantity#70, ss_list_price#71, ss_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, d_date_sk#78] + +(92) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#70, ss_list_price#71, i_brand_id#75, i_class_id#76, i_category_id#77] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#86, isEmpty#87, count#88] + +(93) Exchange +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, 5), ENSURE_REQUIREMENTS, [id=#89] + +(94) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#75, i_class_id#76, i_category_id#77, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [store AS channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#93, count(1)#91 AS number_sales#94] + +(95) Filter [codegen id : 51] +Input [6]: [channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] +Condition : (isnotnull(sales#93) AND (cast(sales#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) + +(96) BroadcastExchange +Input [6]: [channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#95] + +(97) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Right keys [3]: [i_brand_id#75, i_class_id#76, i_category_id#77] Join condition: None -(100) TakeOrderedAndProject -Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +(98) TakeOrderedAndProject +Input [12]: [channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sales#65, number_sales#66, channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#64, i_brand_id#46, i_class_id#47, i_category_id#48, sales#65, number_sales#66, channel#92, i_brand_id#75, i_class_id#76, i_category_id#77, sales#93, number_sales#94] ===== Subqueries ===== Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* HashAggregate (123) -+- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * ColumnarToRow (102) - : : +- Scan parquet default.store_sales (101) - : +- BroadcastExchange (107) - : +- * Project (106) - : +- * Filter (105) - : +- * ColumnarToRow (104) - : +- Scan parquet default.date_dim (103) - :- * Project (114) - : +- * BroadcastHashJoin Inner BuildRight (113) - : :- * ColumnarToRow (111) - : : +- Scan parquet default.catalog_sales (110) - : +- ReusedExchange (112) - +- * Project (119) - +- * BroadcastHashJoin Inner BuildRight (118) - :- * ColumnarToRow (116) - : +- Scan parquet default.web_sales (115) - +- ReusedExchange (117) - - -(101) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +* HashAggregate (121) ++- Exchange (120) + +- * HashAggregate (119) + +- Union (118) + :- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * ColumnarToRow (100) + : : +- Scan parquet default.store_sales (99) + : +- BroadcastExchange (105) + : +- * Project (104) + : +- * Filter (103) + : +- * ColumnarToRow (102) + : +- Scan parquet default.date_dim (101) + :- * Project (112) + : +- * BroadcastHashJoin Inner BuildRight (111) + : :- * ColumnarToRow (109) + : : +- Scan parquet default.catalog_sales (108) + : +- ReusedExchange (110) + +- * Project (117) + +- * BroadcastHashJoin Inner BuildRight (116) + :- * ColumnarToRow (114) + : +- Scan parquet default.web_sales (113) + +- ReusedExchange (115) + + +(99) Scan parquet default.store_sales +Output [3]: [ss_quantity#96, ss_list_price#97, ss_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +(100) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#96, ss_list_price#97, ss_sold_date_sk#98] -(103) Scan parquet default.date_dim -Output [2]: [d_date_sk#102, d_year#103] +(101) Scan parquet default.date_dim +Output [2]: [d_date_sk#100, d_year#101] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] +(102) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#100, d_year#101] -(105) Filter [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#102)) +(103) Filter [codegen id : 1] +Input [2]: [d_date_sk#100, d_year#101] +Condition : (((isnotnull(d_year#101) AND (d_year#101 >= 1998)) AND (d_year#101 <= 2000)) AND isnotnull(d_date_sk#100)) -(106) Project [codegen id : 1] -Output [1]: [d_date_sk#102] -Input [2]: [d_date_sk#102, d_year#103] +(104) Project [codegen id : 1] +Output [1]: [d_date_sk#100] +Input [2]: [d_date_sk#100, d_year#101] -(107) BroadcastExchange -Input [1]: [d_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] +(105) BroadcastExchange +Input [1]: [d_date_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] -(108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +(106) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None -(109) Project [codegen id : 2] -Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] -Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] +(107) Project [codegen id : 2] +Output [2]: [ss_quantity#96 AS quantity#103, ss_list_price#97 AS list_price#104] +Input [4]: [ss_quantity#96, ss_list_price#97, ss_sold_date_sk#98, d_date_sk#100] -(110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +(108) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#105, cs_list_price#106, cs_sold_date_sk#107] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#99)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +(109) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#105, cs_list_price#106, cs_sold_date_sk#107] -(112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#110] +(110) ReusedExchange [Reuses operator id: 105] +Output [1]: [d_date_sk#108] -(113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#109] -Right keys [1]: [d_date_sk#110] +(111) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#107] +Right keys [1]: [d_date_sk#108] Join condition: None -(114) Project [codegen id : 4] -Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] -Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] +(112) Project [codegen id : 4] +Output [2]: [cs_quantity#105 AS quantity#109, cs_list_price#106 AS list_price#110] +Input [4]: [cs_quantity#105, cs_list_price#106, cs_sold_date_sk#107, d_date_sk#108] -(115) Scan parquet default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +(113) Scan parquet default.web_sales +Output [3]: [ws_quantity#111, ws_list_price#112, ws_sold_date_sk#113] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ws_sold_date_sk#113), dynamicpruningexpression(ws_sold_date_sk#113 IN dynamicpruning#99)] ReadSchema: struct -(116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +(114) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#111, ws_list_price#112, ws_sold_date_sk#113] -(117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#116] +(115) ReusedExchange [Reuses operator id: 105] +Output [1]: [d_date_sk#114] -(118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +(116) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#113] +Right keys [1]: [d_date_sk#114] Join condition: None -(119) Project [codegen id : 6] -Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] +(117) Project [codegen id : 6] +Output [2]: [ws_quantity#111 AS quantity#115, ws_list_price#112 AS list_price#116] +Input [4]: [ws_quantity#111, ws_list_price#112, ws_sold_date_sk#113, d_date_sk#114] -(120) Union +(118) Union -(121) HashAggregate [codegen id : 7] -Input [2]: [quantity#105, list_price#106] +(119) HashAggregate [codegen id : 7] +Input [2]: [quantity#103, list_price#104] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#119, count#120] -Results [2]: [sum#121, count#122] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#117, count#118] +Results [2]: [sum#119, count#120] -(122) Exchange -Input [2]: [sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +(120) Exchange +Input [2]: [sum#119, count#120] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#121] -(123) HashAggregate [codegen id : 8] -Input [2]: [sum#121, count#122] +(121) HashAggregate [codegen id : 8] +Input [2]: [sum#119, count#120] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))#122] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#103 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#104 as decimal(12,2)))), DecimalType(18,2), true))#122 AS average_sales#123] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 -ReusedExchange (124) +Subquery:2 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#99 +ReusedExchange (122) -(124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#102] +(122) ReusedExchange [Reuses operator id: 105] +Output [1]: [d_date_sk#100] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 +Subquery:3 Hosting operator id = 108 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#99 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 +Subquery:4 Hosting operator id = 113 Hosting Expression = ws_sold_date_sk#113 IN dynamicpruning#99 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (125) +ReusedExchange (123) -(125) ReusedExchange [Reuses operator id: 70] +(123) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#50] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -ReusedExchange (126) +ReusedExchange (124) -(126) ReusedExchange [Reuses operator id: 26] +(124) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#29] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 @@ -728,62 +718,62 @@ Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN d Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* Project (130) -+- * Filter (129) - +- * ColumnarToRow (128) - +- Scan parquet default.date_dim (127) +* Project (128) ++- * Filter (127) + +- * ColumnarToRow (126) + +- Scan parquet default.date_dim (125) -(127) Scan parquet default.date_dim -Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +(125) Scan parquet default.date_dim +Output [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] 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] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +(126) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] -(129) Filter [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 1999)) AND (d_moy#128 = 12)) AND (d_dom#129 = 16)) +(127) Filter [codegen id : 1] +Input [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] +Condition : (((((isnotnull(d_year#125) AND isnotnull(d_moy#126)) AND isnotnull(d_dom#127)) AND (d_year#125 = 1999)) AND (d_moy#126 = 12)) AND (d_dom#127 = 16)) -(130) Project [codegen id : 1] -Output [1]: [d_week_seq#126] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +(128) Project [codegen id : 1] +Output [1]: [d_week_seq#124] +Input [4]: [d_week_seq#124, d_year#125, d_moy#126, d_dom#127] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +Subquery:10 Hosting operator id = 95 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 -ReusedExchange (131) +Subquery:11 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#72 IN dynamicpruning#73 +ReusedExchange (129) -(131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#79] +(129) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#78] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] -* Project (135) -+- * Filter (134) - +- * ColumnarToRow (133) - +- Scan parquet default.date_dim (132) +Subquery:12 Hosting operator id = 87 Hosting Expression = Subquery scalar-subquery#80, [id=#81] +* Project (133) ++- * Filter (132) + +- * ColumnarToRow (131) + +- Scan parquet default.date_dim (130) -(132) Scan parquet default.date_dim -Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +(130) Scan parquet default.date_dim +Output [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +(131) ColumnarToRow [codegen id : 1] +Input [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] -(134) Filter [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1998)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) +(132) Filter [codegen id : 1] +Input [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] +Condition : (((((isnotnull(d_year#129) AND isnotnull(d_moy#130)) AND isnotnull(d_dom#131)) AND (d_year#129 = 1998)) AND (d_moy#130 = 12)) AND (d_dom#131 = 16)) -(135) Project [codegen id : 1] -Output [1]: [d_week_seq#130] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +(133) Project [codegen id : 1] +Output [1]: [d_week_seq#128] +Input [4]: [d_week_seq#128, d_year#129, d_moy#130, d_dom#131] 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 cb9b410ea856..bfed68434f02 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,213 +1,211 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] WholeStageCodegen (52) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Project [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 #4 - WholeStageCodegen (8) - 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 [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - ReusedExchange [d_date_sk] #14 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #14 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #14 - 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),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_class_id,i_category_id] #1 - WholeStageCodegen (25) - 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_quantity,ss_list_price,ss_sold_date_sk,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] + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + 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 [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - 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] + Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + ReusedExchange [d_date_sk] #14 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #14 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #14 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + 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_quantity,ss_list_price,ss_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + 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_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,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] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - 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_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - 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_year,d_date_sk] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + 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_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - 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] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #12 - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (24) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #3 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - 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] + 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_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + 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] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #12 + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (24) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #3 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] + ColumnarToRow + InputAdapter + 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 #15 WholeStageCodegen (51) - Project [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] #4 - 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),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_class_id,i_category_id] #16 - WholeStageCodegen (50) - 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_quantity,ss_list_price,ss_sold_date_sk,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] + Filter [sales] + ReusedSubquery [average_sales] #4 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (50) + 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_quantity,ss_list_price,ss_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #6 + ReusedExchange [d_date_sk] #17 + InputAdapter + ReusedExchange [ss_item_sk] #12 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (49) + Project [d_date_sk] + Filter [d_week_seq,d_date_sk] + Subquery #7 + WholeStageCodegen (1) + Project [d_week_seq] + Filter [d_year,d_moy,d_dom] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_week_seq,d_year,d_moy,d_dom] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #6 - ReusedExchange [d_date_sk] #17 - InputAdapter - ReusedExchange [ss_item_sk] #12 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (49) - Project [d_date_sk] - Filter [d_week_seq,d_date_sk] - Subquery #7 - WholeStageCodegen (1) - Project [d_week_seq] - Filter [d_year,d_moy,d_dom] - ColumnarToRow - InputAdapter - 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] + 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 499046b0d1f3..37598058110e 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 @@ -1,161 +1,158 @@ == Physical Plan == -TakeOrderedAndProject (157) -+- * HashAggregate (156) - +- Exchange (155) - +- * HashAggregate (154) - +- Union (153) - :- * HashAggregate (132) - : +- Exchange (131) - : +- * HashAggregate (130) - : +- Union (129) - : :- * Project (90) - : : +- * Filter (89) - : : +- * HashAggregate (88) - : : +- Exchange (87) - : : +- * HashAggregate (86) - : : +- * Project (85) - : : +- * BroadcastHashJoin Inner BuildRight (84) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * SortMergeJoin LeftSemi (67) - : : : : :- * Sort (5) - : : : : : +- Exchange (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (66) - : : : : +- Exchange (65) - : : : : +- * Project (64) - : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : :- * Filter (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- Scan parquet default.item (6) - : : : : +- BroadcastExchange (62) - : : : : +- * HashAggregate (61) - : : : : +- Exchange (60) - : : : : +- * HashAggregate (59) - : : : : +- * SortMergeJoin LeftSemi (58) - : : : : :- * Sort (46) - : : : : : +- Exchange (45) - : : : : : +- * HashAggregate (44) - : : : : : +- Exchange (43) - : : : : : +- * HashAggregate (42) - : : : : : +- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (39) - : : : : : +- * SortMergeJoin LeftSemi (38) - : : : : : :- * Sort (23) - : : : : : : +- Exchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (37) - : : : : : +- Exchange (36) - : : : : : +- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (26) - : : : : : : : +- * ColumnarToRow (25) - : : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : : +- ReusedExchange (27) - : : : : : +- BroadcastExchange (33) - : : : : : +- * Filter (32) - : : : : : +- * ColumnarToRow (31) - : : : : : +- Scan parquet default.item (30) - : : : : +- * Sort (57) - : : : : +- Exchange (56) - : : : : +- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Project (52) - : : : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : : : :- * Filter (49) - : : : : : : +- * ColumnarToRow (48) - : : : : : : +- Scan parquet default.web_sales (47) - : : : : : +- ReusedExchange (50) - : : : : +- ReusedExchange (53) - : : : +- BroadcastExchange (72) - : : : +- * Project (71) - : : : +- * Filter (70) - : : : +- * ColumnarToRow (69) - : : : +- Scan parquet default.date_dim (68) - : : +- BroadcastExchange (83) - : : +- * SortMergeJoin LeftSemi (82) - : : :- * Sort (79) - : : : +- Exchange (78) - : : : +- * Filter (77) - : : : +- * ColumnarToRow (76) - : : : +- Scan parquet default.item (75) - : : +- * Sort (81) - : : +- ReusedExchange (80) - : :- * Project (109) - : : +- * Filter (108) - : : +- * HashAggregate (107) - : : +- Exchange (106) - : : +- * HashAggregate (105) - : : +- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Project (101) - : : : +- * BroadcastHashJoin Inner BuildRight (100) - : : : :- * SortMergeJoin LeftSemi (98) - : : : : :- * Sort (95) - : : : : : +- Exchange (94) - : : : : : +- * Filter (93) - : : : : : +- * ColumnarToRow (92) - : : : : : +- Scan parquet default.catalog_sales (91) - : : : : +- * Sort (97) - : : : : +- ReusedExchange (96) - : : : +- ReusedExchange (99) - : : +- ReusedExchange (102) - : +- * Project (128) - : +- * Filter (127) - : +- * HashAggregate (126) - : +- Exchange (125) - : +- * HashAggregate (124) - : +- * Project (123) - : +- * BroadcastHashJoin Inner BuildRight (122) - : :- * Project (120) - : : +- * BroadcastHashJoin Inner BuildRight (119) - : : :- * SortMergeJoin LeftSemi (117) - : : : :- * Sort (114) - : : : : +- Exchange (113) - : : : : +- * Filter (112) - : : : : +- * ColumnarToRow (111) - : : : : +- Scan parquet default.web_sales (110) - : : : +- * Sort (116) - : : : +- ReusedExchange (115) - : : +- ReusedExchange (118) - : +- ReusedExchange (121) - :- * HashAggregate (137) - : +- Exchange (136) - : +- * HashAggregate (135) - : +- * HashAggregate (134) - : +- ReusedExchange (133) - :- * HashAggregate (142) - : +- Exchange (141) - : +- * HashAggregate (140) - : +- * HashAggregate (139) - : +- ReusedExchange (138) - :- * HashAggregate (147) - : +- Exchange (146) - : +- * HashAggregate (145) - : +- * HashAggregate (144) - : +- ReusedExchange (143) - +- * HashAggregate (152) - +- Exchange (151) - +- * HashAggregate (150) - +- * HashAggregate (149) - +- ReusedExchange (148) +TakeOrderedAndProject (154) ++- * HashAggregate (153) + +- Exchange (152) + +- * HashAggregate (151) + +- Union (150) + :- * HashAggregate (129) + : +- Exchange (128) + : +- * HashAggregate (127) + : +- Union (126) + : :- * Filter (89) + : : +- * HashAggregate (88) + : : +- Exchange (87) + : : +- * HashAggregate (86) + : : +- * Project (85) + : : +- * BroadcastHashJoin Inner BuildRight (84) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * SortMergeJoin LeftSemi (67) + : : : : :- * Sort (5) + : : : : : +- Exchange (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- * Sort (66) + : : : : +- Exchange (65) + : : : : +- * Project (64) + : : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : : :- * Filter (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- Scan parquet default.item (6) + : : : : +- BroadcastExchange (62) + : : : : +- * HashAggregate (61) + : : : : +- Exchange (60) + : : : : +- * HashAggregate (59) + : : : : +- * SortMergeJoin LeftSemi (58) + : : : : :- * Sort (46) + : : : : : +- Exchange (45) + : : : : : +- * HashAggregate (44) + : : : : : +- Exchange (43) + : : : : : +- * HashAggregate (42) + : : : : : +- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (39) + : : : : : +- * SortMergeJoin LeftSemi (38) + : : : : : :- * Sort (23) + : : : : : : +- Exchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (37) + : : : : : +- Exchange (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (26) + : : : : : : : +- * ColumnarToRow (25) + : : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : : +- ReusedExchange (27) + : : : : : +- BroadcastExchange (33) + : : : : : +- * Filter (32) + : : : : : +- * ColumnarToRow (31) + : : : : : +- Scan parquet default.item (30) + : : : : +- * Sort (57) + : : : : +- Exchange (56) + : : : : +- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Project (52) + : : : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : : : :- * Filter (49) + : : : : : : +- * ColumnarToRow (48) + : : : : : : +- Scan parquet default.web_sales (47) + : : : : : +- ReusedExchange (50) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (72) + : : : +- * Project (71) + : : : +- * Filter (70) + : : : +- * ColumnarToRow (69) + : : : +- Scan parquet default.date_dim (68) + : : +- BroadcastExchange (83) + : : +- * SortMergeJoin LeftSemi (82) + : : :- * Sort (79) + : : : +- Exchange (78) + : : : +- * Filter (77) + : : : +- * ColumnarToRow (76) + : : : +- Scan parquet default.item (75) + : : +- * Sort (81) + : : +- ReusedExchange (80) + : :- * Filter (107) + : : +- * HashAggregate (106) + : : +- Exchange (105) + : : +- * HashAggregate (104) + : : +- * Project (103) + : : +- * BroadcastHashJoin Inner BuildRight (102) + : : :- * Project (100) + : : : +- * BroadcastHashJoin Inner BuildRight (99) + : : : :- * SortMergeJoin LeftSemi (97) + : : : : :- * Sort (94) + : : : : : +- Exchange (93) + : : : : : +- * Filter (92) + : : : : : +- * ColumnarToRow (91) + : : : : : +- Scan parquet default.catalog_sales (90) + : : : : +- * Sort (96) + : : : : +- ReusedExchange (95) + : : : +- ReusedExchange (98) + : : +- ReusedExchange (101) + : +- * Filter (125) + : +- * HashAggregate (124) + : +- Exchange (123) + : +- * HashAggregate (122) + : +- * Project (121) + : +- * BroadcastHashJoin Inner BuildRight (120) + : :- * Project (118) + : : +- * BroadcastHashJoin Inner BuildRight (117) + : : :- * SortMergeJoin LeftSemi (115) + : : : :- * Sort (112) + : : : : +- Exchange (111) + : : : : +- * Filter (110) + : : : : +- * ColumnarToRow (109) + : : : : +- Scan parquet default.web_sales (108) + : : : +- * Sort (114) + : : : +- ReusedExchange (113) + : : +- ReusedExchange (116) + : +- ReusedExchange (119) + :- * HashAggregate (134) + : +- Exchange (133) + : +- * HashAggregate (132) + : +- * HashAggregate (131) + : +- ReusedExchange (130) + :- * HashAggregate (139) + : +- Exchange (138) + : +- * HashAggregate (137) + : +- * HashAggregate (136) + : +- ReusedExchange (135) + :- * HashAggregate (144) + : +- Exchange (143) + : +- * HashAggregate (142) + : +- * HashAggregate (141) + : +- ReusedExchange (140) + +- * HashAggregate (149) + +- Exchange (148) + +- * HashAggregate (147) + +- * HashAggregate (146) + +- ReusedExchange (145) (1) Scan parquet default.store_sales @@ -559,538 +556,526 @@ Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [6]: [store AS channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#69, count(1)#67 AS number_sales#70] (89) Filter [codegen id : 46] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sales#69, number_sales#70] +Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 46] -Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] - -(91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +(90) Scan parquet default.catalog_sales +Output [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#76), dynamicpruningexpression(cs_sold_date_sk#76 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 47] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +(91) ColumnarToRow [codegen id : 47] +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] -(93) Filter [codegen id : 47] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Condition : isnotnull(cs_item_sk#74) +(92) Filter [codegen id : 47] +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] +Condition : isnotnull(cs_item_sk#73) -(94) Exchange -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] +(93) Exchange +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] +Arguments: hashpartitioning(cs_item_sk#73, 5), ENSURE_REQUIREMENTS, [id=#77] -(95) Sort [codegen id : 48] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 +(94) Sort [codegen id : 48] +Input [4]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76] +Arguments: [cs_item_sk#73 ASC NULLS FIRST], false, 0 -(96) ReusedExchange [Reuses operator id: unknown] +(95) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 67] +(96) Sort [codegen id : 67] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(98) SortMergeJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#74] +(97) SortMergeJoin [codegen id : 91] +Left keys [1]: [cs_item_sk#73] Right keys [1]: [ss_item_sk#47] Join condition: None -(99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#79] +(98) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#78] -(100) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +(99) BroadcastHashJoin [codegen id : 91] +Left keys [1]: [cs_sold_date_sk#76] +Right keys [1]: [d_date_sk#78] Join condition: None -(101) Project [codegen id : 91] -Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] -Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] +(100) Project [codegen id : 91] +Output [3]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75] +Input [5]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, cs_sold_date_sk#76, d_date_sk#78] -(102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +(101) ReusedExchange [Reuses operator id: 83] +Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(103) BroadcastHashJoin [codegen id : 91] -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [i_item_sk#80] +(102) BroadcastHashJoin [codegen id : 91] +Left keys [1]: [cs_item_sk#73] +Right keys [1]: [i_item_sk#79] Join condition: None -(104) Project [codegen id : 91] -Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(105) HashAggregate [codegen id : 91] -Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] - -(106) Exchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] - -(107) HashAggregate [codegen id : 92] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] - -(108) Filter [codegen id : 92] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(109) Project [codegen id : 92] -Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] - -(110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +(103) Project [codegen id : 91] +Output [5]: [cs_quantity#74, cs_list_price#75, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [7]: [cs_item_sk#73, cs_quantity#74, cs_list_price#75, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] + +(104) HashAggregate [codegen id : 91] +Input [5]: [cs_quantity#74, cs_list_price#75, i_brand_id#80, i_class_id#81, i_category_id#82] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] + +(105) Exchange +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] + +(106) HashAggregate [codegen id : 92] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [catalog AS channel#92, i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#74 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#75 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#93, count(1)#91 AS number_sales#94] + +(107) Filter [codegen id : 92] +Input [6]: [channel#92, i_brand_id#80, i_class_id#81, i_category_id#82, sales#93, number_sales#94] +Condition : (isnotnull(sales#93) AND (cast(sales#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) + +(108) Scan parquet default.web_sales +Output [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#98), dynamicpruningexpression(ws_sold_date_sk#98 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 93] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +(109) ColumnarToRow [codegen id : 93] +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] -(112) Filter [codegen id : 93] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Condition : isnotnull(ws_item_sk#97) +(110) Filter [codegen id : 93] +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +Condition : isnotnull(ws_item_sk#95) -(113) Exchange -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] +(111) Exchange +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +Arguments: hashpartitioning(ws_item_sk#95, 5), ENSURE_REQUIREMENTS, [id=#99] -(114) Sort [codegen id : 94] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 +(112) Sort [codegen id : 94] +Input [4]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98] +Arguments: [ws_item_sk#95 ASC NULLS FIRST], false, 0 -(115) ReusedExchange [Reuses operator id: unknown] +(113) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 113] +(114) Sort [codegen id : 113] Input [1]: [ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 -(117) SortMergeJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#97] +(115) SortMergeJoin [codegen id : 137] +Left keys [1]: [ws_item_sk#95] Right keys [1]: [ss_item_sk#47] Join condition: None -(118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#102] +(116) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#100] -(119) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +(117) BroadcastHashJoin [codegen id : 137] +Left keys [1]: [ws_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None -(120) Project [codegen id : 137] -Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] -Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] +(118) Project [codegen id : 137] +Output [3]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97] +Input [5]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, ws_sold_date_sk#98, d_date_sk#100] -(121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +(119) ReusedExchange [Reuses operator id: 83] +Output [4]: [i_item_sk#101, i_brand_id#102, i_class_id#103, i_category_id#104] -(122) BroadcastHashJoin [codegen id : 137] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#103] +(120) BroadcastHashJoin [codegen id : 137] +Left keys [1]: [ws_item_sk#95] +Right keys [1]: [i_item_sk#101] Join condition: None -(123) Project [codegen id : 137] -Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] - -(124) HashAggregate [codegen id : 137] -Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] - -(125) Exchange -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] - -(126) HashAggregate [codegen id : 138] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] - -(127) Filter [codegen id : 138] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) - -(128) Project [codegen id : 138] -Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] - -(129) Union - -(130) HashAggregate [codegen id : 139] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] -Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] - -(131) Exchange -Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] - -(132) HashAggregate [codegen id : 140] -Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] -Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] - -(133) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, sum#136] - -(134) HashAggregate [codegen id : 280] -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, sum#136] -Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#137, sum(number_sales#69)#138] -Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#137 AS sum_sales#129, sum(number_sales#69)#138 AS number_sales#130] - -(135) HashAggregate [codegen id : 280] -Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] -Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#139, isEmpty#140, sum#141] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144] - -(136) Exchange -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#145] - -(137) HashAggregate [codegen id : 281] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#142, isEmpty#143, sum#144] -Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#146, sum(number_sales#130)#147] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#148, sum(sum_sales#129)#146 AS sum(sum_sales)#149, sum(number_sales#130)#147 AS sum(number_sales)#150] - -(138) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] - -(139) HashAggregate [codegen id : 421] -Input [7]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153, sum#154, isEmpty#155, sum#156] -Keys [4]: [channel#73, i_brand_id#151, i_class_id#152, i_category_id#153] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#157, sum(number_sales#69)#158] -Results [4]: [channel#73, i_brand_id#151, sum(sales#68)#157 AS sum_sales#129, sum(number_sales#69)#158 AS number_sales#130] - -(140) HashAggregate [codegen id : 421] -Input [4]: [channel#73, i_brand_id#151, sum_sales#129, number_sales#130] -Keys [2]: [channel#73, i_brand_id#151] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#159, isEmpty#160, sum#161] -Results [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164] - -(141) Exchange -Input [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164] -Arguments: hashpartitioning(channel#73, i_brand_id#151, 5), ENSURE_REQUIREMENTS, [id=#165] - -(142) HashAggregate [codegen id : 422] -Input [5]: [channel#73, i_brand_id#151, sum#162, isEmpty#163, sum#164] -Keys [2]: [channel#73, i_brand_id#151] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#166, sum(number_sales#130)#167] -Results [6]: [channel#73, i_brand_id#151, null AS i_class_id#168, null AS i_category_id#169, sum(sum_sales#129)#166 AS sum(sum_sales)#170, sum(number_sales#130)#167 AS sum(number_sales)#171] - -(143) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174, sum#175, isEmpty#176, sum#177] - -(144) HashAggregate [codegen id : 562] -Input [7]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174, sum#175, isEmpty#176, sum#177] -Keys [4]: [channel#73, i_brand_id#172, i_class_id#173, i_category_id#174] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#178, sum(number_sales#69)#179] -Results [3]: [channel#73, sum(sales#68)#178 AS sum_sales#129, sum(number_sales#69)#179 AS number_sales#130] - -(145) HashAggregate [codegen id : 562] -Input [3]: [channel#73, sum_sales#129, number_sales#130] -Keys [1]: [channel#73] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#180, isEmpty#181, sum#182] -Results [4]: [channel#73, sum#183, isEmpty#184, sum#185] - -(146) Exchange -Input [4]: [channel#73, sum#183, isEmpty#184, sum#185] -Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#186] - -(147) HashAggregate [codegen id : 563] -Input [4]: [channel#73, sum#183, isEmpty#184, sum#185] -Keys [1]: [channel#73] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#187, sum(number_sales#130)#188] -Results [6]: [channel#73, null AS i_brand_id#189, null AS i_class_id#190, null AS i_category_id#191, sum(sum_sales#129)#187 AS sum(sum_sales)#192, sum(number_sales#130)#188 AS sum(number_sales)#193] - -(148) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196, sum#197, isEmpty#198, sum#199] - -(149) HashAggregate [codegen id : 703] -Input [7]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196, sum#197, isEmpty#198, sum#199] -Keys [4]: [channel#73, i_brand_id#194, i_class_id#195, i_category_id#196] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#200, sum(number_sales#69)#201] -Results [2]: [sum(sales#68)#200 AS sum_sales#129, sum(number_sales#69)#201 AS number_sales#130] - -(150) HashAggregate [codegen id : 703] -Input [2]: [sum_sales#129, number_sales#130] +(121) Project [codegen id : 137] +Output [5]: [ws_quantity#96, ws_list_price#97, i_brand_id#102, i_class_id#103, i_category_id#104] +Input [7]: [ws_item_sk#95, ws_quantity#96, ws_list_price#97, i_item_sk#101, i_brand_id#102, i_class_id#103, i_category_id#104] + +(122) HashAggregate [codegen id : 137] +Input [5]: [ws_quantity#96, ws_list_price#97, i_brand_id#102, i_class_id#103, i_category_id#104] +Keys [3]: [i_brand_id#102, i_class_id#103, i_category_id#104] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#105, isEmpty#106, count#107] +Results [6]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum#108, isEmpty#109, count#110] + +(123) Exchange +Input [6]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum#108, isEmpty#109, count#110] +Arguments: hashpartitioning(i_brand_id#102, i_class_id#103, i_category_id#104, 5), ENSURE_REQUIREMENTS, [id=#111] + +(124) HashAggregate [codegen id : 138] +Input [6]: [i_brand_id#102, i_class_id#103, i_category_id#104, sum#108, isEmpty#109, count#110] +Keys [3]: [i_brand_id#102, i_class_id#103, i_category_id#104] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true))#112, count(1)#113] +Results [6]: [web AS channel#114, i_brand_id#102, i_class_id#103, i_category_id#104, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#97 as decimal(12,2)))), DecimalType(18,2), true))#112 AS sales#115, count(1)#113 AS number_sales#116] + +(125) Filter [codegen id : 138] +Input [6]: [channel#114, i_brand_id#102, i_class_id#103, i_category_id#104, sales#115, number_sales#116] +Condition : (isnotnull(sales#115) AND (cast(sales#115 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) + +(126) Union + +(127) HashAggregate [codegen id : 139] +Input [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sales#69, number_sales#70] +Keys [4]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [partial_sum(sales#69), partial_sum(number_sales#70)] +Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] +Results [7]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum#120, isEmpty#121, sum#122] + +(128) Exchange +Input [7]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum#120, isEmpty#121, sum#122] +Arguments: hashpartitioning(channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#123] + +(129) HashAggregate [codegen id : 140] +Input [7]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum#120, isEmpty#121, sum#122] +Keys [4]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [sum(sales#69), sum(number_sales#70)] +Aggregate Attributes [2]: [sum(sales#69)#124, sum(number_sales#70)#125] +Results [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#69)#124 AS sum_sales#126, sum(number_sales#70)#125 AS number_sales#127] + +(130) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#128, i_class_id#129, i_category_id#130, sum#131, isEmpty#132, sum#133] + +(131) HashAggregate [codegen id : 280] +Input [7]: [channel#68, i_brand_id#128, i_class_id#129, i_category_id#130, sum#131, isEmpty#132, sum#133] +Keys [4]: [channel#68, i_brand_id#128, i_class_id#129, i_category_id#130] +Functions [2]: [sum(sales#69), sum(number_sales#70)] +Aggregate Attributes [2]: [sum(sales#69)#134, sum(number_sales#70)#135] +Results [5]: [channel#68, i_brand_id#128, i_class_id#129, sum(sales#69)#134 AS sum_sales#126, sum(number_sales#70)#135 AS number_sales#127] + +(132) HashAggregate [codegen id : 280] +Input [5]: [channel#68, i_brand_id#128, i_class_id#129, sum_sales#126, number_sales#127] +Keys [3]: [channel#68, i_brand_id#128, i_class_id#129] +Functions [2]: [partial_sum(sum_sales#126), partial_sum(number_sales#127)] +Aggregate Attributes [3]: [sum#136, isEmpty#137, sum#138] +Results [6]: [channel#68, i_brand_id#128, i_class_id#129, sum#139, isEmpty#140, sum#141] + +(133) Exchange +Input [6]: [channel#68, i_brand_id#128, i_class_id#129, sum#139, isEmpty#140, sum#141] +Arguments: hashpartitioning(channel#68, i_brand_id#128, i_class_id#129, 5), ENSURE_REQUIREMENTS, [id=#142] + +(134) HashAggregate [codegen id : 281] +Input [6]: [channel#68, i_brand_id#128, i_class_id#129, sum#139, isEmpty#140, sum#141] +Keys [3]: [channel#68, i_brand_id#128, i_class_id#129] +Functions [2]: [sum(sum_sales#126), sum(number_sales#127)] +Aggregate Attributes [2]: [sum(sum_sales#126)#143, sum(number_sales#127)#144] +Results [6]: [channel#68, i_brand_id#128, i_class_id#129, null AS i_category_id#145, sum(sum_sales#126)#143 AS sum(sum_sales)#146, sum(number_sales#127)#144 AS sum(number_sales)#147] + +(135) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#148, i_class_id#149, i_category_id#150, sum#151, isEmpty#152, sum#153] + +(136) HashAggregate [codegen id : 421] +Input [7]: [channel#68, i_brand_id#148, i_class_id#149, i_category_id#150, sum#151, isEmpty#152, sum#153] +Keys [4]: [channel#68, i_brand_id#148, i_class_id#149, i_category_id#150] +Functions [2]: [sum(sales#69), sum(number_sales#70)] +Aggregate Attributes [2]: [sum(sales#69)#154, sum(number_sales#70)#155] +Results [4]: [channel#68, i_brand_id#148, sum(sales#69)#154 AS sum_sales#126, sum(number_sales#70)#155 AS number_sales#127] + +(137) HashAggregate [codegen id : 421] +Input [4]: [channel#68, i_brand_id#148, sum_sales#126, number_sales#127] +Keys [2]: [channel#68, i_brand_id#148] +Functions [2]: [partial_sum(sum_sales#126), partial_sum(number_sales#127)] +Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] +Results [5]: [channel#68, i_brand_id#148, sum#159, isEmpty#160, sum#161] + +(138) Exchange +Input [5]: [channel#68, i_brand_id#148, sum#159, isEmpty#160, sum#161] +Arguments: hashpartitioning(channel#68, i_brand_id#148, 5), ENSURE_REQUIREMENTS, [id=#162] + +(139) HashAggregate [codegen id : 422] +Input [5]: [channel#68, i_brand_id#148, sum#159, isEmpty#160, sum#161] +Keys [2]: [channel#68, i_brand_id#148] +Functions [2]: [sum(sum_sales#126), sum(number_sales#127)] +Aggregate Attributes [2]: [sum(sum_sales#126)#163, sum(number_sales#127)#164] +Results [6]: [channel#68, i_brand_id#148, null AS i_class_id#165, null AS i_category_id#166, sum(sum_sales#126)#163 AS sum(sum_sales)#167, sum(number_sales#127)#164 AS sum(number_sales)#168] + +(140) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#169, i_class_id#170, i_category_id#171, sum#172, isEmpty#173, sum#174] + +(141) HashAggregate [codegen id : 562] +Input [7]: [channel#68, i_brand_id#169, i_class_id#170, i_category_id#171, sum#172, isEmpty#173, sum#174] +Keys [4]: [channel#68, i_brand_id#169, i_class_id#170, i_category_id#171] +Functions [2]: [sum(sales#69), sum(number_sales#70)] +Aggregate Attributes [2]: [sum(sales#69)#175, sum(number_sales#70)#176] +Results [3]: [channel#68, sum(sales#69)#175 AS sum_sales#126, sum(number_sales#70)#176 AS number_sales#127] + +(142) HashAggregate [codegen id : 562] +Input [3]: [channel#68, sum_sales#126, number_sales#127] +Keys [1]: [channel#68] +Functions [2]: [partial_sum(sum_sales#126), partial_sum(number_sales#127)] +Aggregate Attributes [3]: [sum#177, isEmpty#178, sum#179] +Results [4]: [channel#68, sum#180, isEmpty#181, sum#182] + +(143) Exchange +Input [4]: [channel#68, sum#180, isEmpty#181, sum#182] +Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#183] + +(144) HashAggregate [codegen id : 563] +Input [4]: [channel#68, sum#180, isEmpty#181, sum#182] +Keys [1]: [channel#68] +Functions [2]: [sum(sum_sales#126), sum(number_sales#127)] +Aggregate Attributes [2]: [sum(sum_sales#126)#184, sum(number_sales#127)#185] +Results [6]: [channel#68, null AS i_brand_id#186, null AS i_class_id#187, null AS i_category_id#188, sum(sum_sales#126)#184 AS sum(sum_sales)#189, sum(number_sales#127)#185 AS sum(number_sales)#190] + +(145) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#68, i_brand_id#191, i_class_id#192, i_category_id#193, sum#194, isEmpty#195, sum#196] + +(146) HashAggregate [codegen id : 703] +Input [7]: [channel#68, i_brand_id#191, i_class_id#192, i_category_id#193, sum#194, isEmpty#195, sum#196] +Keys [4]: [channel#68, i_brand_id#191, i_class_id#192, i_category_id#193] +Functions [2]: [sum(sales#69), sum(number_sales#70)] +Aggregate Attributes [2]: [sum(sales#69)#197, sum(number_sales#70)#198] +Results [2]: [sum(sales#69)#197 AS sum_sales#126, sum(number_sales#70)#198 AS number_sales#127] + +(147) HashAggregate [codegen id : 703] +Input [2]: [sum_sales#126, number_sales#127] Keys: [] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#202, isEmpty#203, sum#204] -Results [3]: [sum#205, isEmpty#206, sum#207] +Functions [2]: [partial_sum(sum_sales#126), partial_sum(number_sales#127)] +Aggregate Attributes [3]: [sum#199, isEmpty#200, sum#201] +Results [3]: [sum#202, isEmpty#203, sum#204] -(151) Exchange -Input [3]: [sum#205, isEmpty#206, sum#207] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] +(148) Exchange +Input [3]: [sum#202, isEmpty#203, sum#204] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#205] -(152) HashAggregate [codegen id : 704] -Input [3]: [sum#205, isEmpty#206, sum#207] +(149) HashAggregate [codegen id : 704] +Input [3]: [sum#202, isEmpty#203, sum#204] Keys: [] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#209, sum(number_sales#130)#210] -Results [6]: [null AS channel#211, null AS i_brand_id#212, null AS i_class_id#213, null AS i_category_id#214, sum(sum_sales#129)#209 AS sum(sum_sales)#215, sum(number_sales#130)#210 AS sum(number_sales)#216] +Functions [2]: [sum(sum_sales#126), sum(number_sales#127)] +Aggregate Attributes [2]: [sum(sum_sales#126)#206, sum(number_sales#127)#207] +Results [6]: [null AS channel#208, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#126)#206 AS sum(sum_sales)#212, sum(number_sales#127)#207 AS sum(number_sales)#213] -(153) Union +(150) Union -(154) HashAggregate [codegen id : 705] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +(151) HashAggregate [codegen id : 705] +Input [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] +Keys [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Results [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] -(155) Exchange -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#217] +(152) Exchange +Input [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] +Arguments: hashpartitioning(channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127, 5), ENSURE_REQUIREMENTS, [id=#214] -(156) HashAggregate [codegen id : 706] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +(153) HashAggregate [codegen id : 706] +Input [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] +Keys [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Results [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] -(157) TakeOrderedAndProject -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +(154) TakeOrderedAndProject +Input [6]: [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] +Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#68, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#126, number_sales#127] ===== Subqueries ===== Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] -* HashAggregate (184) -+- Exchange (183) - +- * HashAggregate (182) - +- Union (181) - :- * Project (166) - : +- * BroadcastHashJoin Inner BuildRight (165) - : :- * ColumnarToRow (159) - : : +- Scan parquet default.store_sales (158) - : +- BroadcastExchange (164) - : +- * Project (163) - : +- * Filter (162) - : +- * ColumnarToRow (161) - : +- Scan parquet default.date_dim (160) - :- * Project (175) - : +- * BroadcastHashJoin Inner BuildRight (174) - : :- * ColumnarToRow (168) - : : +- Scan parquet default.catalog_sales (167) - : +- BroadcastExchange (173) - : +- * Project (172) - : +- * Filter (171) - : +- * ColumnarToRow (170) - : +- Scan parquet default.date_dim (169) - +- * Project (180) - +- * BroadcastHashJoin Inner BuildRight (179) - :- * ColumnarToRow (177) - : +- Scan parquet default.web_sales (176) - +- ReusedExchange (178) - - -(158) Scan parquet default.store_sales -Output [3]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220] +* HashAggregate (181) ++- Exchange (180) + +- * HashAggregate (179) + +- Union (178) + :- * Project (163) + : +- * BroadcastHashJoin Inner BuildRight (162) + : :- * ColumnarToRow (156) + : : +- Scan parquet default.store_sales (155) + : +- BroadcastExchange (161) + : +- * Project (160) + : +- * Filter (159) + : +- * ColumnarToRow (158) + : +- Scan parquet default.date_dim (157) + :- * Project (172) + : +- * BroadcastHashJoin Inner BuildRight (171) + : :- * ColumnarToRow (165) + : : +- Scan parquet default.catalog_sales (164) + : +- BroadcastExchange (170) + : +- * Project (169) + : +- * Filter (168) + : +- * ColumnarToRow (167) + : +- Scan parquet default.date_dim (166) + +- * Project (177) + +- * BroadcastHashJoin Inner BuildRight (176) + :- * ColumnarToRow (174) + : +- Scan parquet default.web_sales (173) + +- ReusedExchange (175) + + +(155) Scan parquet default.store_sales +Output [3]: [ss_quantity#215, ss_list_price#216, ss_sold_date_sk#217] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#220), dynamicpruningexpression(ss_sold_date_sk#220 IN dynamicpruning#221)] +PartitionFilters: [isnotnull(ss_sold_date_sk#217), dynamicpruningexpression(ss_sold_date_sk#217 IN dynamicpruning#218)] ReadSchema: struct -(159) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220] +(156) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#215, ss_list_price#216, ss_sold_date_sk#217] -(160) Scan parquet default.date_dim -Output [2]: [d_date_sk#222, d_year#223] +(157) Scan parquet default.date_dim +Output [2]: [d_date_sk#219, d_year#220] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(161) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#222, d_year#223] +(158) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#219, d_year#220] -(162) Filter [codegen id : 1] -Input [2]: [d_date_sk#222, d_year#223] -Condition : (((isnotnull(d_year#223) AND (d_year#223 >= 1999)) AND (d_year#223 <= 2001)) AND isnotnull(d_date_sk#222)) +(159) Filter [codegen id : 1] +Input [2]: [d_date_sk#219, d_year#220] +Condition : (((isnotnull(d_year#220) AND (d_year#220 >= 1999)) AND (d_year#220 <= 2001)) AND isnotnull(d_date_sk#219)) -(163) Project [codegen id : 1] -Output [1]: [d_date_sk#222] -Input [2]: [d_date_sk#222, d_year#223] +(160) Project [codegen id : 1] +Output [1]: [d_date_sk#219] +Input [2]: [d_date_sk#219, d_year#220] -(164) BroadcastExchange -Input [1]: [d_date_sk#222] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#224] +(161) BroadcastExchange +Input [1]: [d_date_sk#219] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#221] -(165) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#220] -Right keys [1]: [d_date_sk#222] +(162) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#217] +Right keys [1]: [d_date_sk#219] Join condition: None -(166) Project [codegen id : 2] -Output [2]: [ss_quantity#218 AS quantity#225, ss_list_price#219 AS list_price#226] -Input [4]: [ss_quantity#218, ss_list_price#219, ss_sold_date_sk#220, d_date_sk#222] +(163) Project [codegen id : 2] +Output [2]: [ss_quantity#215 AS quantity#222, ss_list_price#216 AS list_price#223] +Input [4]: [ss_quantity#215, ss_list_price#216, ss_sold_date_sk#217, d_date_sk#219] -(167) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229] +(164) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#224, cs_list_price#225, cs_sold_date_sk#226] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#229), dynamicpruningexpression(cs_sold_date_sk#229 IN dynamicpruning#230)] +PartitionFilters: [isnotnull(cs_sold_date_sk#226), dynamicpruningexpression(cs_sold_date_sk#226 IN dynamicpruning#227)] ReadSchema: struct -(168) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229] +(165) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#224, cs_list_price#225, cs_sold_date_sk#226] -(169) Scan parquet default.date_dim -Output [2]: [d_date_sk#231, d_year#232] +(166) Scan parquet default.date_dim +Output [2]: [d_date_sk#228, d_year#229] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(170) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#231, d_year#232] +(167) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#228, d_year#229] -(171) Filter [codegen id : 3] -Input [2]: [d_date_sk#231, d_year#232] -Condition : (((isnotnull(d_year#232) AND (d_year#232 >= 1998)) AND (d_year#232 <= 2000)) AND isnotnull(d_date_sk#231)) +(168) Filter [codegen id : 3] +Input [2]: [d_date_sk#228, d_year#229] +Condition : (((isnotnull(d_year#229) AND (d_year#229 >= 1998)) AND (d_year#229 <= 2000)) AND isnotnull(d_date_sk#228)) -(172) Project [codegen id : 3] -Output [1]: [d_date_sk#231] -Input [2]: [d_date_sk#231, d_year#232] +(169) Project [codegen id : 3] +Output [1]: [d_date_sk#228] +Input [2]: [d_date_sk#228, d_year#229] -(173) BroadcastExchange -Input [1]: [d_date_sk#231] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#233] +(170) BroadcastExchange +Input [1]: [d_date_sk#228] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#230] -(174) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#229] -Right keys [1]: [d_date_sk#231] +(171) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#226] +Right keys [1]: [d_date_sk#228] Join condition: None -(175) Project [codegen id : 4] -Output [2]: [cs_quantity#227 AS quantity#234, cs_list_price#228 AS list_price#235] -Input [4]: [cs_quantity#227, cs_list_price#228, cs_sold_date_sk#229, d_date_sk#231] +(172) Project [codegen id : 4] +Output [2]: [cs_quantity#224 AS quantity#231, cs_list_price#225 AS list_price#232] +Input [4]: [cs_quantity#224, cs_list_price#225, cs_sold_date_sk#226, d_date_sk#228] -(176) Scan parquet default.web_sales -Output [3]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238] +(173) Scan parquet default.web_sales +Output [3]: [ws_quantity#233, ws_list_price#234, ws_sold_date_sk#235] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#238), dynamicpruningexpression(ws_sold_date_sk#238 IN dynamicpruning#230)] +PartitionFilters: [isnotnull(ws_sold_date_sk#235), dynamicpruningexpression(ws_sold_date_sk#235 IN dynamicpruning#227)] ReadSchema: struct -(177) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238] +(174) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#233, ws_list_price#234, ws_sold_date_sk#235] -(178) ReusedExchange [Reuses operator id: 173] -Output [1]: [d_date_sk#239] +(175) ReusedExchange [Reuses operator id: 170] +Output [1]: [d_date_sk#236] -(179) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#238] -Right keys [1]: [d_date_sk#239] +(176) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#235] +Right keys [1]: [d_date_sk#236] Join condition: None -(180) Project [codegen id : 6] -Output [2]: [ws_quantity#236 AS quantity#240, ws_list_price#237 AS list_price#241] -Input [4]: [ws_quantity#236, ws_list_price#237, ws_sold_date_sk#238, d_date_sk#239] +(177) Project [codegen id : 6] +Output [2]: [ws_quantity#233 AS quantity#237, ws_list_price#234 AS list_price#238] +Input [4]: [ws_quantity#233, ws_list_price#234, ws_sold_date_sk#235, d_date_sk#236] -(181) Union +(178) Union -(182) HashAggregate [codegen id : 7] -Input [2]: [quantity#225, list_price#226] +(179) HashAggregate [codegen id : 7] +Input [2]: [quantity#222, list_price#223] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#242, count#243] -Results [2]: [sum#244, count#245] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#222 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#223 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#239, count#240] +Results [2]: [sum#241, count#242] -(183) Exchange -Input [2]: [sum#244, count#245] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#246] +(180) Exchange +Input [2]: [sum#241, count#242] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#243] -(184) HashAggregate [codegen id : 8] -Input [2]: [sum#244, count#245] +(181) HashAggregate [codegen id : 8] +Input [2]: [sum#241, count#242] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))#247] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#225 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#226 as decimal(12,2)))), DecimalType(18,2), true))#247 AS average_sales#248] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#222 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#223 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#222 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#223 as decimal(12,2)))), DecimalType(18,2), true))#244] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#222 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#223 as decimal(12,2)))), DecimalType(18,2), true))#244 AS average_sales#245] -Subquery:2 Hosting operator id = 158 Hosting Expression = ss_sold_date_sk#220 IN dynamicpruning#221 -ReusedExchange (185) +Subquery:2 Hosting operator id = 155 Hosting Expression = ss_sold_date_sk#217 IN dynamicpruning#218 +ReusedExchange (182) -(185) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#222] +(182) ReusedExchange [Reuses operator id: 161] +Output [1]: [d_date_sk#219] -Subquery:3 Hosting operator id = 167 Hosting Expression = cs_sold_date_sk#229 IN dynamicpruning#230 -ReusedExchange (186) +Subquery:3 Hosting operator id = 164 Hosting Expression = cs_sold_date_sk#226 IN dynamicpruning#227 +ReusedExchange (183) -(186) ReusedExchange [Reuses operator id: 173] -Output [1]: [d_date_sk#231] +(183) ReusedExchange [Reuses operator id: 170] +Output [1]: [d_date_sk#228] -Subquery:4 Hosting operator id = 176 Hosting Expression = ws_sold_date_sk#238 IN dynamicpruning#230 +Subquery:4 Hosting operator id = 173 Hosting Expression = ws_sold_date_sk#235 IN dynamicpruning#227 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (187) +ReusedExchange (184) -(187) ReusedExchange [Reuses operator id: 72] +(184) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#49] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -ReusedExchange (188) +ReusedExchange (185) -(188) ReusedExchange [Reuses operator id: 16] +(185) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#14] Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:9 Hosting operator id = 107 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#76 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:11 Hosting operator id = 125 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 108 Hosting Expression = ws_sold_date_sk#98 IN dynamicpruning#5 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 6a4204810175..ff0705992594 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 @@ -16,282 +16,279 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (46) - Project [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 #3 - WholeStageCodegen (8) - 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 [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - ReusedExchange [d_date_sk] #21 + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + 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 [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #21 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - ReusedExchange [d_date_sk] #22 + Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + ReusedExchange [d_date_sk] #21 + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #22 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #5 + Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + ReusedExchange [d_date_sk] #22 + InputAdapter + BroadcastExchange #22 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #22 - 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),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_class_id,i_category_id] #3 - WholeStageCodegen (45) - 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] - SortMergeJoin [ss_item_sk,ss_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (1) - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - InputAdapter - WholeStageCodegen (21) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #6 - WholeStageCodegen (20) - Project [i_item_sk] - 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_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (19) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (18) - HashAggregate [brand_id,class_id,category_id] - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (13) - Sort [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (12) - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (11) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (10) - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (8) - 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] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (17) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (16) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (22) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (44) - SortMergeJoin [i_item_sk,ss_item_sk] + Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #5 InputAdapter - WholeStageCodegen (24) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #18 - WholeStageCodegen (23) - Filter [i_item_sk] + ReusedExchange [d_date_sk] #22 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (45) + 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] + SortMergeJoin [ss_item_sk,ss_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + InputAdapter + WholeStageCodegen (21) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #6 + WholeStageCodegen (20) + Project [i_item_sk] + 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_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - WholeStageCodegen (43) - Sort [ss_item_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (19) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #8 + WholeStageCodegen (18) + HashAggregate [brand_id,class_id,category_id] + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (13) + Sort [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (12) + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #10 + WholeStageCodegen (11) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (10) + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (8) + 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] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + WholeStageCodegen (17) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (16) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (22) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (92) - Project [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] #3 - 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),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_class_id,i_category_id] #23 - WholeStageCodegen (91) - 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_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - SortMergeJoin [cs_item_sk,ss_item_sk] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (44) + SortMergeJoin [i_item_sk,ss_item_sk] InputAdapter - WholeStageCodegen (48) - Sort [cs_item_sk] + WholeStageCodegen (24) + Sort [i_item_sk] InputAdapter - Exchange [cs_item_sk] #24 - WholeStageCodegen (47) - Filter [cs_item_sk] + Exchange [i_item_sk] #18 + WholeStageCodegen (23) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - WholeStageCodegen (67) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 + WholeStageCodegen (92) + Filter [sales] + ReusedSubquery [average_sales] #3 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #23 + WholeStageCodegen (91) + 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_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + SortMergeJoin [cs_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (48) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #24 + WholeStageCodegen (47) + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (67) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk] #19 + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 WholeStageCodegen (138) - Project [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] #3 - 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),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_class_id,i_category_id] #25 - WholeStageCodegen (137) - 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] - SortMergeJoin [ws_item_sk,ss_item_sk] - InputAdapter - WholeStageCodegen (94) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #26 - WholeStageCodegen (93) - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - WholeStageCodegen (113) - Sort [ss_item_sk] - InputAdapter - ReusedExchange [ss_item_sk] #19 + Filter [sales] + ReusedSubquery [average_sales] #3 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #25 + WholeStageCodegen (137) + 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] + SortMergeJoin [ws_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (94) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #26 + WholeStageCodegen (93) + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (113) + Sort [ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk] #19 + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 WholeStageCodegen (281) 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 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 844ae4b303d7..9ba3e814e1ce 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 @@ -1,142 +1,139 @@ == Physical Plan == -TakeOrderedAndProject (138) -+- * HashAggregate (137) - +- Exchange (136) - +- * HashAggregate (135) - +- Union (134) - :- * HashAggregate (113) - : +- Exchange (112) - : +- * HashAggregate (111) - : +- Union (110) - : :- * Project (77) - : : +- * Filter (76) - : : +- * HashAggregate (75) - : : +- Exchange (74) - : : +- * HashAggregate (73) - : : +- * Project (72) - : : +- * BroadcastHashJoin Inner BuildRight (71) - : : :- * Project (65) - : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : :- * Filter (6) - : : : : : +- * ColumnarToRow (5) - : : : : : +- Scan parquet default.item (4) - : : : : +- BroadcastExchange (53) - : : : : +- * HashAggregate (52) - : : : : +- * HashAggregate (51) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : : : : :- * HashAggregate (39) - : : : : : +- Exchange (38) - : : : : : +- * HashAggregate (37) - : : : : : +- * Project (36) - : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : :- * Project (33) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (31) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * Project (25) - : : : : : : +- * Filter (24) - : : : : : : +- * ColumnarToRow (23) - : : : : : : +- Scan parquet default.date_dim (22) - : : : : : +- ReusedExchange (34) - : : : : +- BroadcastExchange (49) - : : : : +- * Project (48) - : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : :- * Project (45) - : : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : : :- * Filter (42) - : : : : : : +- * ColumnarToRow (41) - : : : : : : +- Scan parquet default.web_sales (40) - : : : : : +- ReusedExchange (43) - : : : : +- ReusedExchange (46) - : : : +- BroadcastExchange (63) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (62) - : : : :- * Filter (60) - : : : : +- * ColumnarToRow (59) - : : : : +- Scan parquet default.item (58) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (70) - : : +- * Project (69) - : : +- * Filter (68) - : : +- * ColumnarToRow (67) - : : +- Scan parquet default.date_dim (66) - : :- * Project (93) - : : +- * Filter (92) - : : +- * HashAggregate (91) - : : +- Exchange (90) - : : +- * HashAggregate (89) - : : +- * Project (88) - : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * Project (85) - : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (82) - : : : : :- * Filter (80) - : : : : : +- * ColumnarToRow (79) - : : : : : +- Scan parquet default.catalog_sales (78) - : : : : +- ReusedExchange (81) - : : : +- ReusedExchange (83) - : : +- ReusedExchange (86) - : +- * Project (109) - : +- * Filter (108) - : +- * HashAggregate (107) - : +- Exchange (106) - : +- * HashAggregate (105) - : +- * Project (104) - : +- * BroadcastHashJoin Inner BuildRight (103) - : :- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * BroadcastHashJoin LeftSemi BuildRight (98) - : : : :- * Filter (96) - : : : : +- * ColumnarToRow (95) - : : : : +- Scan parquet default.web_sales (94) - : : : +- ReusedExchange (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - :- * HashAggregate (118) - : +- Exchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- ReusedExchange (114) - :- * HashAggregate (123) - : +- Exchange (122) - : +- * HashAggregate (121) - : +- * HashAggregate (120) - : +- ReusedExchange (119) - :- * HashAggregate (128) - : +- Exchange (127) - : +- * HashAggregate (126) - : +- * HashAggregate (125) - : +- ReusedExchange (124) - +- * HashAggregate (133) - +- Exchange (132) - +- * HashAggregate (131) - +- * HashAggregate (130) - +- ReusedExchange (129) +TakeOrderedAndProject (135) ++- * HashAggregate (134) + +- Exchange (133) + +- * HashAggregate (132) + +- Union (131) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- Union (107) + : :- * Filter (76) + : : +- * HashAggregate (75) + : : +- Exchange (74) + : : +- * HashAggregate (73) + : : +- * Project (72) + : : +- * BroadcastHashJoin Inner BuildRight (71) + : : :- * Project (65) + : : : +- * BroadcastHashJoin Inner BuildRight (64) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (57) + : : : : :- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- BroadcastExchange (56) + : : : : +- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.item (4) + : : : : +- BroadcastExchange (53) + : : : : +- * HashAggregate (52) + : : : : +- * HashAggregate (51) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : : : : :- * HashAggregate (39) + : : : : : +- Exchange (38) + : : : : : +- * HashAggregate (37) + : : : : : +- * Project (36) + : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : :- * Project (33) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (31) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * Project (25) + : : : : : : +- * Filter (24) + : : : : : : +- * ColumnarToRow (23) + : : : : : : +- Scan parquet default.date_dim (22) + : : : : : +- ReusedExchange (34) + : : : : +- BroadcastExchange (49) + : : : : +- * Project (48) + : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : :- * Project (45) + : : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : : :- * Filter (42) + : : : : : : +- * ColumnarToRow (41) + : : : : : : +- Scan parquet default.web_sales (40) + : : : : : +- ReusedExchange (43) + : : : : +- ReusedExchange (46) + : : : +- BroadcastExchange (63) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (62) + : : : :- * Filter (60) + : : : : +- * ColumnarToRow (59) + : : : : +- Scan parquet default.item (58) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (70) + : : +- * Project (69) + : : +- * Filter (68) + : : +- * ColumnarToRow (67) + : : +- Scan parquet default.date_dim (66) + : :- * Filter (91) + : : +- * HashAggregate (90) + : : +- Exchange (89) + : : +- * HashAggregate (88) + : : +- * Project (87) + : : +- * BroadcastHashJoin Inner BuildRight (86) + : : :- * Project (84) + : : : +- * BroadcastHashJoin Inner BuildRight (83) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (81) + : : : : :- * Filter (79) + : : : : : +- * ColumnarToRow (78) + : : : : : +- Scan parquet default.catalog_sales (77) + : : : : +- ReusedExchange (80) + : : : +- ReusedExchange (82) + : : +- ReusedExchange (85) + : +- * Filter (106) + : +- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- * Project (102) + : +- * BroadcastHashJoin Inner BuildRight (101) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * BroadcastHashJoin LeftSemi BuildRight (96) + : : : :- * Filter (94) + : : : : +- * ColumnarToRow (93) + : : : : +- Scan parquet default.web_sales (92) + : : : +- ReusedExchange (95) + : : +- ReusedExchange (97) + : +- ReusedExchange (100) + :- * HashAggregate (115) + : +- Exchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- ReusedExchange (111) + :- * HashAggregate (120) + : +- Exchange (119) + : +- * HashAggregate (118) + : +- * HashAggregate (117) + : +- ReusedExchange (116) + :- * HashAggregate (125) + : +- Exchange (124) + : +- * HashAggregate (123) + : +- * HashAggregate (122) + : +- ReusedExchange (121) + +- * HashAggregate (130) + +- Exchange (129) + +- * HashAggregate (128) + +- * HashAggregate (127) + +- ReusedExchange (126) (1) Scan parquet default.store_sales @@ -488,514 +485,502 @@ Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Results [6]: [store AS channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#64, count(1)#62 AS number_sales#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Condition : (isnotnull(sales#64) AND (cast(sales#64 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) -(77) Project [codegen id : 26] -Output [6]: [store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] - -(78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +(77) Scan parquet default.catalog_sales +Output [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#71), dynamicpruningexpression(cs_sold_date_sk#71 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +(78) ColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] -(80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#69) +(79) Filter [codegen id : 51] +Input [4]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71] +Condition : isnotnull(cs_item_sk#68) -(81) ReusedExchange [Reuses operator id: unknown] +(80) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#43] -(82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] +(81) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#68] Right keys [1]: [ss_item_sk#43] Join condition: None -(83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +(82) ReusedExchange [Reuses operator id: 63] +Output [4]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75] -(84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [i_item_sk#73] +(83) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#68] +Right keys [1]: [i_item_sk#72] Join condition: None -(85) Project [codegen id : 51] -Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +(84) Project [codegen id : 51] +Output [6]: [cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, i_brand_id#73, i_class_id#74, i_category_id#75] +Input [8]: [cs_item_sk#68, cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75] -(86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#77] +(85) ReusedExchange [Reuses operator id: 70] +Output [1]: [d_date_sk#76] -(87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#72] -Right keys [1]: [d_date_sk#77] +(86) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#71] +Right keys [1]: [d_date_sk#76] Join condition: None -(88) Project [codegen id : 51] -Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] - -(89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] - -(90) Exchange -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] - -(91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] - -(92) Filter [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(93) Project [codegen id : 52] -Output [6]: [catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] - -(94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(87) Project [codegen id : 51] +Output [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#73, i_class_id#74, i_category_id#75] +Input [7]: [cs_quantity#69, cs_list_price#70, cs_sold_date_sk#71, i_brand_id#73, i_class_id#74, i_category_id#75, d_date_sk#76] + +(88) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#69, cs_list_price#70, i_brand_id#73, i_class_id#74, i_category_id#75] +Keys [3]: [i_brand_id#73, i_class_id#74, i_category_id#75] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum#80, isEmpty#81, count#82] + +(89) Exchange +Input [6]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, 5), ENSURE_REQUIREMENTS, [id=#83] + +(90) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#73, i_class_id#74, i_category_id#75, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#73, i_class_id#74, i_category_id#75] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] +Results [6]: [catalog AS channel#86, i_brand_id#73, i_class_id#74, i_category_id#75, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#87, count(1)#85 AS number_sales#88] + +(91) Filter [codegen id : 52] +Input [6]: [channel#86, i_brand_id#73, i_class_id#74, i_category_id#75, sales#87, number_sales#88] +Condition : (isnotnull(sales#87) AND (cast(sales#87 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) + +(92) Scan parquet default.web_sales +Output [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(93) ColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] -(96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -Condition : isnotnull(ws_item_sk#91) +(94) Filter [codegen id : 77] +Input [4]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_item_sk#89) -(97) ReusedExchange [Reuses operator id: unknown] +(95) ReusedExchange [Reuses operator id: unknown] Output [1]: [ss_item_sk#43] -(98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] +(96) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#89] Right keys [1]: [ss_item_sk#43] Join condition: None -(99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +(97) ReusedExchange [Reuses operator id: 63] +Output [4]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96] -(100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [i_item_sk#95] +(98) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#89] +Right keys [1]: [i_item_sk#93] Join condition: None -(101) Project [codegen id : 77] -Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +(99) Project [codegen id : 77] +Output [6]: [ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, i_brand_id#94, i_class_id#95, i_category_id#96] +Input [8]: [ws_item_sk#89, ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96] -(102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#99] +(100) ReusedExchange [Reuses operator id: 70] +Output [1]: [d_date_sk#97] -(103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#99] +(101) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#97] Join condition: None -(104) Project [codegen id : 77] -Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] - -(105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] - -(106) Exchange -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] - -(107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] - -(108) Filter [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) - -(109) Project [codegen id : 78] -Output [6]: [web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] - -(110) Union - -(111) HashAggregate [codegen id : 79] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] -Results [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] - -(112) Exchange -Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#119] - -(113) HashAggregate [codegen id : 80] -Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] -Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] - -(114) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, sum#129] - -(115) HashAggregate [codegen id : 160] -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, sum#129] -Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#130, sum(number_sales#64)#131] -Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#130 AS sum_sales#122, sum(number_sales#64)#131 AS number_sales#123] - -(116) HashAggregate [codegen id : 160] -Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] -Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#132, isEmpty#133, sum#134] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137] - -(117) Exchange -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#138] - -(118) HashAggregate [codegen id : 161] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#135, isEmpty#136, sum#137] -Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#139, sum(number_sales#123)#140] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#141, sum(sum_sales#122)#139 AS sum(sum_sales)#142, sum(number_sales#123)#140 AS sum(number_sales)#143] - -(119) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146, sum#147, isEmpty#148, sum#149] - -(120) HashAggregate [codegen id : 241] -Input [7]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146, sum#147, isEmpty#148, sum#149] -Keys [4]: [channel#68, i_brand_id#144, i_class_id#145, i_category_id#146] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#150, sum(number_sales#64)#151] -Results [4]: [channel#68, i_brand_id#144, sum(sales#63)#150 AS sum_sales#122, sum(number_sales#64)#151 AS number_sales#123] - -(121) HashAggregate [codegen id : 241] -Input [4]: [channel#68, i_brand_id#144, sum_sales#122, number_sales#123] -Keys [2]: [channel#68, i_brand_id#144] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] -Results [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157] - -(122) Exchange -Input [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157] -Arguments: hashpartitioning(channel#68, i_brand_id#144, 5), ENSURE_REQUIREMENTS, [id=#158] - -(123) HashAggregate [codegen id : 242] -Input [5]: [channel#68, i_brand_id#144, sum#155, isEmpty#156, sum#157] -Keys [2]: [channel#68, i_brand_id#144] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#159, sum(number_sales#123)#160] -Results [6]: [channel#68, i_brand_id#144, null AS i_class_id#161, null AS i_category_id#162, sum(sum_sales#122)#159 AS sum(sum_sales)#163, sum(number_sales#123)#160 AS sum(number_sales)#164] - -(124) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167, sum#168, isEmpty#169, sum#170] - -(125) HashAggregate [codegen id : 322] -Input [7]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167, sum#168, isEmpty#169, sum#170] -Keys [4]: [channel#68, i_brand_id#165, i_class_id#166, i_category_id#167] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#171, sum(number_sales#64)#172] -Results [3]: [channel#68, sum(sales#63)#171 AS sum_sales#122, sum(number_sales#64)#172 AS number_sales#123] - -(126) HashAggregate [codegen id : 322] -Input [3]: [channel#68, sum_sales#122, number_sales#123] -Keys [1]: [channel#68] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] -Results [4]: [channel#68, sum#176, isEmpty#177, sum#178] - -(127) Exchange -Input [4]: [channel#68, sum#176, isEmpty#177, sum#178] -Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#179] - -(128) HashAggregate [codegen id : 323] -Input [4]: [channel#68, sum#176, isEmpty#177, sum#178] -Keys [1]: [channel#68] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#180, sum(number_sales#123)#181] -Results [6]: [channel#68, null AS i_brand_id#182, null AS i_class_id#183, null AS i_category_id#184, sum(sum_sales#122)#180 AS sum(sum_sales)#185, sum(number_sales#123)#181 AS sum(number_sales)#186] - -(129) ReusedExchange [Reuses operator id: unknown] -Output [7]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189, sum#190, isEmpty#191, sum#192] - -(130) HashAggregate [codegen id : 403] -Input [7]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189, sum#190, isEmpty#191, sum#192] -Keys [4]: [channel#68, i_brand_id#187, i_class_id#188, i_category_id#189] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#193, sum(number_sales#64)#194] -Results [2]: [sum(sales#63)#193 AS sum_sales#122, sum(number_sales#64)#194 AS number_sales#123] - -(131) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#122, number_sales#123] +(102) Project [codegen id : 77] +Output [5]: [ws_quantity#90, ws_list_price#91, i_brand_id#94, i_class_id#95, i_category_id#96] +Input [7]: [ws_quantity#90, ws_list_price#91, ws_sold_date_sk#92, i_brand_id#94, i_class_id#95, i_category_id#96, d_date_sk#97] + +(103) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#90, ws_list_price#91, i_brand_id#94, i_class_id#95, i_category_id#96] +Keys [3]: [i_brand_id#94, i_class_id#95, i_category_id#96] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#98, isEmpty#99, count#100] +Results [6]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum#101, isEmpty#102, count#103] + +(104) Exchange +Input [6]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum#101, isEmpty#102, count#103] +Arguments: hashpartitioning(i_brand_id#94, i_class_id#95, i_category_id#96, 5), ENSURE_REQUIREMENTS, [id=#104] + +(105) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#94, i_class_id#95, i_category_id#96, sum#101, isEmpty#102, count#103] +Keys [3]: [i_brand_id#94, i_class_id#95, i_category_id#96] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true))#105, count(1)#106] +Results [6]: [web AS channel#107, i_brand_id#94, i_class_id#95, i_category_id#96, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#90 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#91 as decimal(12,2)))), DecimalType(18,2), true))#105 AS sales#108, count(1)#106 AS number_sales#109] + +(106) Filter [codegen id : 78] +Input [6]: [channel#107, i_brand_id#94, i_class_id#95, i_category_id#96, sales#108, number_sales#109] +Condition : (isnotnull(sales#108) AND (cast(sales#108 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) + +(107) Union + +(108) HashAggregate [codegen id : 79] +Input [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Keys [4]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [partial_sum(sales#64), partial_sum(number_sales#65)] +Aggregate Attributes [3]: [sum#110, isEmpty#111, sum#112] +Results [7]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum#113, isEmpty#114, sum#115] + +(109) Exchange +Input [7]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum#113, isEmpty#114, sum#115] +Arguments: hashpartitioning(channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#116] + +(110) HashAggregate [codegen id : 80] +Input [7]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum#113, isEmpty#114, sum#115] +Keys [4]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [sum(sales#64), sum(number_sales#65)] +Aggregate Attributes [2]: [sum(sales#64)#117, sum(number_sales#65)#118] +Results [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#64)#117 AS sum_sales#119, sum(number_sales#65)#118 AS number_sales#120] + +(111) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#63, i_brand_id#121, i_class_id#122, i_category_id#123, sum#124, isEmpty#125, sum#126] + +(112) HashAggregate [codegen id : 160] +Input [7]: [channel#63, i_brand_id#121, i_class_id#122, i_category_id#123, sum#124, isEmpty#125, sum#126] +Keys [4]: [channel#63, i_brand_id#121, i_class_id#122, i_category_id#123] +Functions [2]: [sum(sales#64), sum(number_sales#65)] +Aggregate Attributes [2]: [sum(sales#64)#127, sum(number_sales#65)#128] +Results [5]: [channel#63, i_brand_id#121, i_class_id#122, sum(sales#64)#127 AS sum_sales#119, sum(number_sales#65)#128 AS number_sales#120] + +(113) HashAggregate [codegen id : 160] +Input [5]: [channel#63, i_brand_id#121, i_class_id#122, sum_sales#119, number_sales#120] +Keys [3]: [channel#63, i_brand_id#121, i_class_id#122] +Functions [2]: [partial_sum(sum_sales#119), partial_sum(number_sales#120)] +Aggregate Attributes [3]: [sum#129, isEmpty#130, sum#131] +Results [6]: [channel#63, i_brand_id#121, i_class_id#122, sum#132, isEmpty#133, sum#134] + +(114) Exchange +Input [6]: [channel#63, i_brand_id#121, i_class_id#122, sum#132, isEmpty#133, sum#134] +Arguments: hashpartitioning(channel#63, i_brand_id#121, i_class_id#122, 5), ENSURE_REQUIREMENTS, [id=#135] + +(115) HashAggregate [codegen id : 161] +Input [6]: [channel#63, i_brand_id#121, i_class_id#122, sum#132, isEmpty#133, sum#134] +Keys [3]: [channel#63, i_brand_id#121, i_class_id#122] +Functions [2]: [sum(sum_sales#119), sum(number_sales#120)] +Aggregate Attributes [2]: [sum(sum_sales#119)#136, sum(number_sales#120)#137] +Results [6]: [channel#63, i_brand_id#121, i_class_id#122, null AS i_category_id#138, sum(sum_sales#119)#136 AS sum(sum_sales)#139, sum(number_sales#120)#137 AS sum(number_sales)#140] + +(116) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#63, i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, sum#146] + +(117) HashAggregate [codegen id : 241] +Input [7]: [channel#63, i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, sum#146] +Keys [4]: [channel#63, i_brand_id#141, i_class_id#142, i_category_id#143] +Functions [2]: [sum(sales#64), sum(number_sales#65)] +Aggregate Attributes [2]: [sum(sales#64)#147, sum(number_sales#65)#148] +Results [4]: [channel#63, i_brand_id#141, sum(sales#64)#147 AS sum_sales#119, sum(number_sales#65)#148 AS number_sales#120] + +(118) HashAggregate [codegen id : 241] +Input [4]: [channel#63, i_brand_id#141, sum_sales#119, number_sales#120] +Keys [2]: [channel#63, i_brand_id#141] +Functions [2]: [partial_sum(sum_sales#119), partial_sum(number_sales#120)] +Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] +Results [5]: [channel#63, i_brand_id#141, sum#152, isEmpty#153, sum#154] + +(119) Exchange +Input [5]: [channel#63, i_brand_id#141, sum#152, isEmpty#153, sum#154] +Arguments: hashpartitioning(channel#63, i_brand_id#141, 5), ENSURE_REQUIREMENTS, [id=#155] + +(120) HashAggregate [codegen id : 242] +Input [5]: [channel#63, i_brand_id#141, sum#152, isEmpty#153, sum#154] +Keys [2]: [channel#63, i_brand_id#141] +Functions [2]: [sum(sum_sales#119), sum(number_sales#120)] +Aggregate Attributes [2]: [sum(sum_sales#119)#156, sum(number_sales#120)#157] +Results [6]: [channel#63, i_brand_id#141, null AS i_class_id#158, null AS i_category_id#159, sum(sum_sales#119)#156 AS sum(sum_sales)#160, sum(number_sales#120)#157 AS sum(number_sales)#161] + +(121) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#63, i_brand_id#162, i_class_id#163, i_category_id#164, sum#165, isEmpty#166, sum#167] + +(122) HashAggregate [codegen id : 322] +Input [7]: [channel#63, i_brand_id#162, i_class_id#163, i_category_id#164, sum#165, isEmpty#166, sum#167] +Keys [4]: [channel#63, i_brand_id#162, i_class_id#163, i_category_id#164] +Functions [2]: [sum(sales#64), sum(number_sales#65)] +Aggregate Attributes [2]: [sum(sales#64)#168, sum(number_sales#65)#169] +Results [3]: [channel#63, sum(sales#64)#168 AS sum_sales#119, sum(number_sales#65)#169 AS number_sales#120] + +(123) HashAggregate [codegen id : 322] +Input [3]: [channel#63, sum_sales#119, number_sales#120] +Keys [1]: [channel#63] +Functions [2]: [partial_sum(sum_sales#119), partial_sum(number_sales#120)] +Aggregate Attributes [3]: [sum#170, isEmpty#171, sum#172] +Results [4]: [channel#63, sum#173, isEmpty#174, sum#175] + +(124) Exchange +Input [4]: [channel#63, sum#173, isEmpty#174, sum#175] +Arguments: hashpartitioning(channel#63, 5), ENSURE_REQUIREMENTS, [id=#176] + +(125) HashAggregate [codegen id : 323] +Input [4]: [channel#63, sum#173, isEmpty#174, sum#175] +Keys [1]: [channel#63] +Functions [2]: [sum(sum_sales#119), sum(number_sales#120)] +Aggregate Attributes [2]: [sum(sum_sales#119)#177, sum(number_sales#120)#178] +Results [6]: [channel#63, null AS i_brand_id#179, null AS i_class_id#180, null AS i_category_id#181, sum(sum_sales#119)#177 AS sum(sum_sales)#182, sum(number_sales#120)#178 AS sum(number_sales)#183] + +(126) ReusedExchange [Reuses operator id: unknown] +Output [7]: [channel#63, i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, sum#189] + +(127) HashAggregate [codegen id : 403] +Input [7]: [channel#63, i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, sum#189] +Keys [4]: [channel#63, i_brand_id#184, i_class_id#185, i_category_id#186] +Functions [2]: [sum(sales#64), sum(number_sales#65)] +Aggregate Attributes [2]: [sum(sales#64)#190, sum(number_sales#65)#191] +Results [2]: [sum(sales#64)#190 AS sum_sales#119, sum(number_sales#65)#191 AS number_sales#120] + +(128) HashAggregate [codegen id : 403] +Input [2]: [sum_sales#119, number_sales#120] Keys: [] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#195, isEmpty#196, sum#197] -Results [3]: [sum#198, isEmpty#199, sum#200] +Functions [2]: [partial_sum(sum_sales#119), partial_sum(number_sales#120)] +Aggregate Attributes [3]: [sum#192, isEmpty#193, sum#194] +Results [3]: [sum#195, isEmpty#196, sum#197] -(132) Exchange -Input [3]: [sum#198, isEmpty#199, sum#200] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#201] +(129) Exchange +Input [3]: [sum#195, isEmpty#196, sum#197] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#198] -(133) HashAggregate [codegen id : 404] -Input [3]: [sum#198, isEmpty#199, sum#200] +(130) HashAggregate [codegen id : 404] +Input [3]: [sum#195, isEmpty#196, sum#197] Keys: [] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#202, sum(number_sales#123)#203] -Results [6]: [null AS channel#204, null AS i_brand_id#205, null AS i_class_id#206, null AS i_category_id#207, sum(sum_sales#122)#202 AS sum(sum_sales)#208, sum(number_sales#123)#203 AS sum(number_sales)#209] +Functions [2]: [sum(sum_sales#119), sum(number_sales#120)] +Aggregate Attributes [2]: [sum(sum_sales#119)#199, sum(number_sales#120)#200] +Results [6]: [null AS channel#201, null AS i_brand_id#202, null AS i_class_id#203, null AS i_category_id#204, sum(sum_sales#119)#199 AS sum(sum_sales)#205, sum(number_sales#120)#200 AS sum(number_sales)#206] -(134) Union +(131) Union -(135) HashAggregate [codegen id : 405] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +(132) HashAggregate [codegen id : 405] +Input [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] +Keys [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Results [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] -(136) Exchange -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#210] +(133) Exchange +Input [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] +Arguments: hashpartitioning(channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120, 5), ENSURE_REQUIREMENTS, [id=#207] -(137) HashAggregate [codegen id : 406] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +(134) HashAggregate [codegen id : 406] +Input [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] +Keys [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Results [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] -(138) TakeOrderedAndProject -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +(135) TakeOrderedAndProject +Input [6]: [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] +Arguments: 100, [channel#63 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#63, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#119, number_sales#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* HashAggregate (165) -+- Exchange (164) - +- * HashAggregate (163) - +- Union (162) - :- * Project (147) - : +- * BroadcastHashJoin Inner BuildRight (146) - : :- * ColumnarToRow (140) - : : +- Scan parquet default.store_sales (139) - : +- BroadcastExchange (145) - : +- * Project (144) - : +- * Filter (143) - : +- * ColumnarToRow (142) - : +- Scan parquet default.date_dim (141) - :- * Project (156) - : +- * BroadcastHashJoin Inner BuildRight (155) - : :- * ColumnarToRow (149) - : : +- Scan parquet default.catalog_sales (148) - : +- BroadcastExchange (154) - : +- * Project (153) - : +- * Filter (152) - : +- * ColumnarToRow (151) - : +- Scan parquet default.date_dim (150) - +- * Project (161) - +- * BroadcastHashJoin Inner BuildRight (160) - :- * ColumnarToRow (158) - : +- Scan parquet default.web_sales (157) - +- ReusedExchange (159) - - -(139) Scan parquet default.store_sales -Output [3]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213] +* HashAggregate (162) ++- Exchange (161) + +- * HashAggregate (160) + +- Union (159) + :- * Project (144) + : +- * BroadcastHashJoin Inner BuildRight (143) + : :- * ColumnarToRow (137) + : : +- Scan parquet default.store_sales (136) + : +- BroadcastExchange (142) + : +- * Project (141) + : +- * Filter (140) + : +- * ColumnarToRow (139) + : +- Scan parquet default.date_dim (138) + :- * Project (153) + : +- * BroadcastHashJoin Inner BuildRight (152) + : :- * ColumnarToRow (146) + : : +- Scan parquet default.catalog_sales (145) + : +- BroadcastExchange (151) + : +- * Project (150) + : +- * Filter (149) + : +- * ColumnarToRow (148) + : +- Scan parquet default.date_dim (147) + +- * Project (158) + +- * BroadcastHashJoin Inner BuildRight (157) + :- * ColumnarToRow (155) + : +- Scan parquet default.web_sales (154) + +- ReusedExchange (156) + + +(136) Scan parquet default.store_sales +Output [3]: [ss_quantity#208, ss_list_price#209, ss_sold_date_sk#210] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#213), dynamicpruningexpression(ss_sold_date_sk#213 IN dynamicpruning#214)] +PartitionFilters: [isnotnull(ss_sold_date_sk#210), dynamicpruningexpression(ss_sold_date_sk#210 IN dynamicpruning#211)] ReadSchema: struct -(140) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213] +(137) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#208, ss_list_price#209, ss_sold_date_sk#210] -(141) Scan parquet default.date_dim -Output [2]: [d_date_sk#215, d_year#216] +(138) Scan parquet default.date_dim +Output [2]: [d_date_sk#212, d_year#213] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(142) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#215, d_year#216] +(139) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#212, d_year#213] -(143) Filter [codegen id : 1] -Input [2]: [d_date_sk#215, d_year#216] -Condition : (((isnotnull(d_year#216) AND (d_year#216 >= 1999)) AND (d_year#216 <= 2001)) AND isnotnull(d_date_sk#215)) +(140) Filter [codegen id : 1] +Input [2]: [d_date_sk#212, d_year#213] +Condition : (((isnotnull(d_year#213) AND (d_year#213 >= 1999)) AND (d_year#213 <= 2001)) AND isnotnull(d_date_sk#212)) -(144) Project [codegen id : 1] -Output [1]: [d_date_sk#215] -Input [2]: [d_date_sk#215, d_year#216] +(141) Project [codegen id : 1] +Output [1]: [d_date_sk#212] +Input [2]: [d_date_sk#212, d_year#213] -(145) BroadcastExchange -Input [1]: [d_date_sk#215] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#217] +(142) BroadcastExchange +Input [1]: [d_date_sk#212] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#214] -(146) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#213] -Right keys [1]: [d_date_sk#215] +(143) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#210] +Right keys [1]: [d_date_sk#212] Join condition: None -(147) Project [codegen id : 2] -Output [2]: [ss_quantity#211 AS quantity#218, ss_list_price#212 AS list_price#219] -Input [4]: [ss_quantity#211, ss_list_price#212, ss_sold_date_sk#213, d_date_sk#215] +(144) Project [codegen id : 2] +Output [2]: [ss_quantity#208 AS quantity#215, ss_list_price#209 AS list_price#216] +Input [4]: [ss_quantity#208, ss_list_price#209, ss_sold_date_sk#210, d_date_sk#212] -(148) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222] +(145) Scan parquet default.catalog_sales +Output [3]: [cs_quantity#217, cs_list_price#218, cs_sold_date_sk#219] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#222), dynamicpruningexpression(cs_sold_date_sk#222 IN dynamicpruning#223)] +PartitionFilters: [isnotnull(cs_sold_date_sk#219), dynamicpruningexpression(cs_sold_date_sk#219 IN dynamicpruning#220)] ReadSchema: struct -(149) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222] +(146) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#217, cs_list_price#218, cs_sold_date_sk#219] -(150) Scan parquet default.date_dim -Output [2]: [d_date_sk#224, d_year#225] +(147) Scan parquet default.date_dim +Output [2]: [d_date_sk#221, d_year#222] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#224, d_year#225] +(148) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#221, d_year#222] -(152) Filter [codegen id : 3] -Input [2]: [d_date_sk#224, d_year#225] -Condition : (((isnotnull(d_year#225) AND (d_year#225 >= 1998)) AND (d_year#225 <= 2000)) AND isnotnull(d_date_sk#224)) +(149) Filter [codegen id : 3] +Input [2]: [d_date_sk#221, d_year#222] +Condition : (((isnotnull(d_year#222) AND (d_year#222 >= 1998)) AND (d_year#222 <= 2000)) AND isnotnull(d_date_sk#221)) -(153) Project [codegen id : 3] -Output [1]: [d_date_sk#224] -Input [2]: [d_date_sk#224, d_year#225] +(150) Project [codegen id : 3] +Output [1]: [d_date_sk#221] +Input [2]: [d_date_sk#221, d_year#222] -(154) BroadcastExchange -Input [1]: [d_date_sk#224] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#226] +(151) BroadcastExchange +Input [1]: [d_date_sk#221] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#223] -(155) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#222] -Right keys [1]: [d_date_sk#224] +(152) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#219] +Right keys [1]: [d_date_sk#221] Join condition: None -(156) Project [codegen id : 4] -Output [2]: [cs_quantity#220 AS quantity#227, cs_list_price#221 AS list_price#228] -Input [4]: [cs_quantity#220, cs_list_price#221, cs_sold_date_sk#222, d_date_sk#224] +(153) Project [codegen id : 4] +Output [2]: [cs_quantity#217 AS quantity#224, cs_list_price#218 AS list_price#225] +Input [4]: [cs_quantity#217, cs_list_price#218, cs_sold_date_sk#219, d_date_sk#221] -(157) Scan parquet default.web_sales -Output [3]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231] +(154) Scan parquet default.web_sales +Output [3]: [ws_quantity#226, ws_list_price#227, ws_sold_date_sk#228] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#231), dynamicpruningexpression(ws_sold_date_sk#231 IN dynamicpruning#223)] +PartitionFilters: [isnotnull(ws_sold_date_sk#228), dynamicpruningexpression(ws_sold_date_sk#228 IN dynamicpruning#220)] ReadSchema: struct -(158) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231] +(155) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#226, ws_list_price#227, ws_sold_date_sk#228] -(159) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#232] +(156) ReusedExchange [Reuses operator id: 151] +Output [1]: [d_date_sk#229] -(160) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#231] -Right keys [1]: [d_date_sk#232] +(157) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#228] +Right keys [1]: [d_date_sk#229] Join condition: None -(161) Project [codegen id : 6] -Output [2]: [ws_quantity#229 AS quantity#233, ws_list_price#230 AS list_price#234] -Input [4]: [ws_quantity#229, ws_list_price#230, ws_sold_date_sk#231, d_date_sk#232] +(158) Project [codegen id : 6] +Output [2]: [ws_quantity#226 AS quantity#230, ws_list_price#227 AS list_price#231] +Input [4]: [ws_quantity#226, ws_list_price#227, ws_sold_date_sk#228, d_date_sk#229] -(162) Union +(159) Union -(163) HashAggregate [codegen id : 7] -Input [2]: [quantity#218, list_price#219] +(160) HashAggregate [codegen id : 7] +Input [2]: [quantity#215, list_price#216] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#235, count#236] -Results [2]: [sum#237, count#238] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#215 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#216 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#232, count#233] +Results [2]: [sum#234, count#235] -(164) Exchange -Input [2]: [sum#237, count#238] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#239] +(161) Exchange +Input [2]: [sum#234, count#235] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#236] -(165) HashAggregate [codegen id : 8] -Input [2]: [sum#237, count#238] +(162) HashAggregate [codegen id : 8] +Input [2]: [sum#234, count#235] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))#240] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#218 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#219 as decimal(12,2)))), DecimalType(18,2), true))#240 AS average_sales#241] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#215 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#216 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#215 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#216 as decimal(12,2)))), DecimalType(18,2), true))#237] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#215 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#216 as decimal(12,2)))), DecimalType(18,2), true))#237 AS average_sales#238] -Subquery:2 Hosting operator id = 139 Hosting Expression = ss_sold_date_sk#213 IN dynamicpruning#214 -ReusedExchange (166) +Subquery:2 Hosting operator id = 136 Hosting Expression = ss_sold_date_sk#210 IN dynamicpruning#211 +ReusedExchange (163) -(166) ReusedExchange [Reuses operator id: 145] -Output [1]: [d_date_sk#215] +(163) ReusedExchange [Reuses operator id: 142] +Output [1]: [d_date_sk#212] -Subquery:3 Hosting operator id = 148 Hosting Expression = cs_sold_date_sk#222 IN dynamicpruning#223 -ReusedExchange (167) +Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#219 IN dynamicpruning#220 +ReusedExchange (164) -(167) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#224] +(164) ReusedExchange [Reuses operator id: 151] +Output [1]: [d_date_sk#221] -Subquery:4 Hosting operator id = 157 Hosting Expression = ws_sold_date_sk#231 IN dynamicpruning#223 +Subquery:4 Hosting operator id = 154 Hosting Expression = ws_sold_date_sk#228 IN dynamicpruning#220 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (168) +ReusedExchange (165) -(168) ReusedExchange [Reuses operator id: 70] +(165) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#50] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -ReusedExchange (169) +ReusedExchange (166) -(169) ReusedExchange [Reuses operator id: 26] +(166) ReusedExchange [Reuses operator id: 26] Output [1]: [d_date_sk#29] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 91 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 77 Hosting Expression = cs_sold_date_sk#71 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 106 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#5 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 521e6d5a776e..e77ffb44b96f 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 @@ -16,225 +16,222 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter Union WholeStageCodegen (26) - Project [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 #3 - WholeStageCodegen (8) - 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 [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - ReusedExchange [d_date_sk] #16 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - ReusedExchange [d_date_sk] #17 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #5 + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + 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 [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #17 - 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),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_class_id,i_category_id] #3 - WholeStageCodegen (25) - 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_quantity,ss_list_price,ss_sold_date_sk,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] + Scan parquet default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + ReusedExchange [d_date_sk] #16 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - 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] + Scan parquet default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + ReusedExchange [d_date_sk] #17 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) + Scan parquet default.date_dim [d_date_sk,d_year] + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #17 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (25) + 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_quantity,ss_list_price,ss_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + Project [i_item_sk] + 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_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (10) + HashAggregate [brand_id,class_id,category_id] HashAggregate [brand_id,class_id,category_id] - HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - HashAggregate [brand_id,class_id,category_id] - InputAdapter - Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,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] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - 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_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,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] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + 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_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - 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] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #14 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (24) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (52) - Project [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] #3 - 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),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_class_id,i_category_id] #18 - WholeStageCodegen (51) - 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_quantity,cs_list_price,cs_sold_date_sk,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] + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + 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] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #14 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (24) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #14 + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (52) + Filter [sales] + ReusedSubquery [average_sales] #3 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (51) + 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_quantity,cs_list_price,cs_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [ss_item_sk] #14 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 WholeStageCodegen (78) - Project [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] #3 - 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),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_class_id,i_category_id] #19 - WholeStageCodegen (77) - 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_quantity,ws_list_price,ws_sold_date_sk,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] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #14 + Filter [sales] + ReusedSubquery [average_sales] #3 + 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,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #19 + WholeStageCodegen (77) + 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_quantity,ws_list_price,ws_sold_date_sk,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] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [ss_item_sk] #14 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 WholeStageCodegen (161) 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 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 7cdc8074e74e..f8c7b2f148ca 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 @@ -1,53 +1,52 @@ == Physical Plan == -* Sort (49) -+- Exchange (48) - +- * Project (47) - +- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * SortMergeJoin Inner (38) - :- * Sort (31) - : +- Exchange (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildLeft (22) - : : :- BroadcastExchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildLeft (15) - : : : :- BroadcastExchange (11) - : : : : +- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : :- BroadcastExchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store (1) - : : : : +- * Filter (8) - : : : : +- * ColumnarToRow (7) - : : : : +- Scan parquet default.customer_address (6) - : : : +- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.customer (12) - : : +- * Project (21) - : : +- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet default.store_sales (18) - : +- BroadcastExchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.item (24) - +- * Sort (37) - +- Exchange (36) - +- * Project (35) - +- * Filter (34) - +- * ColumnarToRow (33) - +- Scan parquet default.store_returns (32) +* Sort (48) ++- Exchange (47) + +- * Filter (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * SortMergeJoin Inner (38) + :- * Sort (31) + : +- Exchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildLeft (22) + : : :- BroadcastExchange (17) + : : : +- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildLeft (15) + : : : :- BroadcastExchange (11) + : : : : +- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : : :- BroadcastExchange (5) + : : : : : +- * Project (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store (1) + : : : : +- * Filter (8) + : : : : +- * ColumnarToRow (7) + : : : : +- Scan parquet default.customer_address (6) + : : : +- * Filter (14) + : : : +- * ColumnarToRow (13) + : : : +- Scan parquet default.customer (12) + : : +- * Project (21) + : : +- * Filter (20) + : : +- * ColumnarToRow (19) + : : +- Scan parquet default.store_sales (18) + : +- BroadcastExchange (27) + : +- * Filter (26) + : +- * ColumnarToRow (25) + : +- Scan parquet default.item (24) + +- * Sort (37) + +- Exchange (36) + +- * Project (35) + +- * Filter (34) + +- * ColumnarToRow (33) + +- Scan parquet default.store_returns (32) (1) Scan parquet default.store @@ -257,296 +256,292 @@ Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum#43, isEmpty#44] Keys [3]: [c_last_name#15, c_first_name#14, s_store_name#2] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#46] -Results [5]: [c_last_name#15, c_first_name#14, s_store_name#2, sum(netpaid#40)#46 AS paid#47, sum(netpaid#40)#46 AS sum(netpaid#40)#48] +Results [4]: [c_last_name#15, c_first_name#14, s_store_name#2, sum(netpaid#40)#46 AS paid#47] (46) Filter [codegen id : 11] -Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47, sum(netpaid#40)#48] -Condition : (isnotnull(sum(netpaid#40)#48) AND (cast(sum(netpaid#40)#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(47) Project [codegen id : 11] -Output [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] -Input [5]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47, sum(netpaid#40)#48] +Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] +Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) -(48) Exchange +(47) Exchange Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] -Arguments: rangepartitioning(c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#51] +Arguments: rangepartitioning(c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#50] -(49) Sort [codegen id : 12] +(48) Sort [codegen id : 12] Input [4]: [c_last_name#15, c_first_name#14, s_store_name#2, paid#47] Arguments: [c_last_name#15 ASC NULLS FIRST, c_first_name#14 ASC NULLS FIRST, s_store_name#2 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (97) -+- Exchange (96) - +- * HashAggregate (95) - +- * HashAggregate (94) - +- Exchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * SortMergeJoin Inner (90) - :- * Sort (83) - : +- Exchange (82) - : +- * Project (81) - : +- * SortMergeJoin Inner (80) - : :- * Sort (74) - : : +- Exchange (73) - : : +- * Project (72) - : : +- * BroadcastHashJoin Inner BuildLeft (71) - : : :- BroadcastExchange (66) - : : : +- * Project (65) - : : : +- * BroadcastHashJoin Inner BuildLeft (64) - : : : :- BroadcastExchange (60) - : : : : +- * Project (59) - : : : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : : : :- BroadcastExchange (54) - : : : : : +- * Project (53) - : : : : : +- * Filter (52) - : : : : : +- * ColumnarToRow (51) - : : : : : +- Scan parquet default.store (50) - : : : : +- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet default.customer_address (55) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Project (70) - : : +- * Filter (69) - : : +- * ColumnarToRow (68) - : : +- Scan parquet default.store_sales (67) - : +- * Sort (79) - : +- Exchange (78) - : +- * Filter (77) - : +- * ColumnarToRow (76) - : +- Scan parquet default.item (75) - +- * Sort (89) - +- Exchange (88) - +- * Project (87) - +- * Filter (86) - +- * ColumnarToRow (85) - +- Scan parquet default.store_returns (84) - - -(50) Scan parquet default.store -Output [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (96) ++- Exchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * Project (90) + +- * SortMergeJoin Inner (89) + :- * Sort (82) + : +- Exchange (81) + : +- * Project (80) + : +- * SortMergeJoin Inner (79) + : :- * Sort (73) + : : +- Exchange (72) + : : +- * Project (71) + : : +- * BroadcastHashJoin Inner BuildLeft (70) + : : :- BroadcastExchange (65) + : : : +- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildLeft (63) + : : : :- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * BroadcastHashJoin Inner BuildLeft (57) + : : : : :- BroadcastExchange (53) + : : : : : +- * Project (52) + : : : : : +- * Filter (51) + : : : : : +- * ColumnarToRow (50) + : : : : : +- Scan parquet default.store (49) + : : : : +- * Filter (56) + : : : : +- * ColumnarToRow (55) + : : : : +- Scan parquet default.customer_address (54) + : : : +- * Filter (62) + : : : +- * ColumnarToRow (61) + : : : +- Scan parquet default.customer (60) + : : +- * Project (69) + : : +- * Filter (68) + : : +- * ColumnarToRow (67) + : : +- Scan parquet default.store_sales (66) + : +- * Sort (78) + : +- Exchange (77) + : +- * Filter (76) + : +- * ColumnarToRow (75) + : +- Scan parquet default.item (74) + +- * Sort (88) + +- Exchange (87) + +- * Project (86) + +- * Filter (85) + +- * ColumnarToRow (84) + +- Scan parquet default.store_returns (83) + + +(49) Scan parquet default.store +Output [5]: [s_store_sk#51, s_store_name#52, s_market_id#53, s_state#54, s_zip#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +(50) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#51, s_store_name#52, s_market_id#53, s_state#54, s_zip#55] -(52) Filter [codegen id : 1] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] -Condition : (((isnotnull(s_market_id#54) AND (s_market_id#54 = 8)) AND isnotnull(s_store_sk#52)) AND isnotnull(s_zip#56)) +(51) Filter [codegen id : 1] +Input [5]: [s_store_sk#51, s_store_name#52, s_market_id#53, s_state#54, s_zip#55] +Condition : (((isnotnull(s_market_id#53) AND (s_market_id#53 = 8)) AND isnotnull(s_store_sk#51)) AND isnotnull(s_zip#55)) -(53) Project [codegen id : 1] -Output [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +(52) Project [codegen id : 1] +Output [4]: [s_store_sk#51, s_store_name#52, s_state#54, s_zip#55] +Input [5]: [s_store_sk#51, s_store_name#52, s_market_id#53, s_state#54, s_zip#55] -(54) BroadcastExchange -Input [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#57] +(53) BroadcastExchange +Input [4]: [s_store_sk#51, s_store_name#52, s_state#54, s_zip#55] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#56] -(55) Scan parquet default.customer_address -Output [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +(54) Scan parquet default.customer_address +Output [4]: [ca_address_sk#57, ca_state#58, ca_zip#59, ca_country#60] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(56) ColumnarToRow -Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +(55) ColumnarToRow +Input [4]: [ca_address_sk#57, ca_state#58, ca_zip#59, ca_country#60] -(57) Filter -Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] -Condition : ((isnotnull(ca_address_sk#58) AND isnotnull(ca_country#61)) AND isnotnull(ca_zip#60)) +(56) Filter +Input [4]: [ca_address_sk#57, ca_state#58, ca_zip#59, ca_country#60] +Condition : ((isnotnull(ca_address_sk#57) AND isnotnull(ca_country#60)) AND isnotnull(ca_zip#59)) -(58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#56] -Right keys [1]: [ca_zip#60] +(57) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [s_zip#55] +Right keys [1]: [ca_zip#59] Join condition: None -(59) Project [codegen id : 2] -Output [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] -Input [8]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56, ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +(58) Project [codegen id : 2] +Output [6]: [s_store_sk#51, s_store_name#52, s_state#54, ca_address_sk#57, ca_state#58, ca_country#60] +Input [8]: [s_store_sk#51, s_store_name#52, s_state#54, s_zip#55, ca_address_sk#57, ca_state#58, ca_zip#59, ca_country#60] -(60) BroadcastExchange -Input [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#62] +(59) BroadcastExchange +Input [6]: [s_store_sk#51, s_store_name#52, s_state#54, ca_address_sk#57, ca_state#58, ca_country#60] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#61] -(61) Scan parquet default.customer -Output [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +(60) Scan parquet default.customer +Output [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(62) ColumnarToRow -Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +(61) ColumnarToRow +Input [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -(63) Filter -Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] -Condition : ((isnotnull(c_customer_sk#63) AND isnotnull(c_current_addr_sk#64)) AND isnotnull(c_birth_country#67)) +(62) Filter +Input [5]: [c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : ((isnotnull(c_customer_sk#62) AND isnotnull(c_current_addr_sk#63)) AND isnotnull(c_birth_country#66)) -(64) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#58, upper(ca_country#61)] -Right keys [2]: [c_current_addr_sk#64, c_birth_country#67] +(63) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [ca_address_sk#57, upper(ca_country#60)] +Right keys [2]: [c_current_addr_sk#63, c_birth_country#66] Join condition: None -(65) Project [codegen id : 3] -Output [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] -Input [11]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61, c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +(64) Project [codegen id : 3] +Output [7]: [s_store_sk#51, s_store_name#52, s_state#54, ca_state#58, c_customer_sk#62, c_first_name#64, c_last_name#65] +Input [11]: [s_store_sk#51, s_store_name#52, s_state#54, ca_address_sk#57, ca_state#58, ca_country#60, c_customer_sk#62, c_current_addr_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -(66) BroadcastExchange -Input [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#68] +(65) BroadcastExchange +Input [7]: [s_store_sk#51, s_store_name#52, s_state#54, ca_state#58, c_customer_sk#62, c_first_name#64, c_last_name#65] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#67] -(67) Scan parquet default.store_sales -Output [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +(66) Scan parquet default.store_sales +Output [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(68) ColumnarToRow -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +(67) ColumnarToRow +Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] -(69) Filter -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] -Condition : (((isnotnull(ss_ticket_number#72) AND isnotnull(ss_item_sk#69)) AND isnotnull(ss_store_sk#71)) AND isnotnull(ss_customer_sk#70)) +(68) Filter +Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] +Condition : (((isnotnull(ss_ticket_number#71) AND isnotnull(ss_item_sk#68)) AND isnotnull(ss_store_sk#70)) AND isnotnull(ss_customer_sk#69)) -(70) Project -Output [5]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +(69) Project +Output [5]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72] +Input [6]: [ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72, ss_sold_date_sk#73] -(71) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#52, c_customer_sk#63] -Right keys [2]: [ss_store_sk#71, ss_customer_sk#70] +(70) BroadcastHashJoin [codegen id : 4] +Left keys [2]: [s_store_sk#51, c_customer_sk#62] +Right keys [2]: [ss_store_sk#70, ss_customer_sk#69] Join condition: None -(72) Project [codegen id : 4] -Output [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Input [12]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] +(71) Project [codegen id : 4] +Output [8]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] +Input [12]: [s_store_sk#51, s_store_name#52, s_state#54, ca_state#58, c_customer_sk#62, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_customer_sk#69, ss_store_sk#70, ss_ticket_number#71, ss_net_paid#72] -(73) Exchange -Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Arguments: hashpartitioning(ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +(72) Exchange +Input [8]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] +Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#74] -(74) Sort [codegen id : 5] -Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Arguments: [ss_item_sk#69 ASC NULLS FIRST], false, 0 +(73) Sort [codegen id : 5] +Input [8]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72] +Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 -(75) Scan parquet default.item -Output [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +(74) Scan parquet default.item +Output [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +(75) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -(77) Filter [codegen id : 6] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Condition : isnotnull(i_item_sk#76) +(76) Filter [codegen id : 6] +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Condition : isnotnull(i_item_sk#75) -(78) Exchange -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: hashpartitioning(i_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +(77) Exchange +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: hashpartitioning(i_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] -(79) Sort [codegen id : 7] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: [i_item_sk#76 ASC NULLS FIRST], false, 0 +(78) Sort [codegen id : 7] +Input [6]: [i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: [i_item_sk#75 ASC NULLS FIRST], false, 0 -(80) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#69] -Right keys [1]: [i_item_sk#76] +(79) SortMergeJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [i_item_sk#75] Join condition: None -(81) Project [codegen id : 8] -Output [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Input [14]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +(80) Project [codegen id : 8] +Output [13]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Input [14]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_item_sk#75, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] -(82) Exchange -Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: hashpartitioning(ss_ticket_number#72, ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#83] +(81) Exchange +Input [13]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: hashpartitioning(ss_ticket_number#71, ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#82] -(83) Sort [codegen id : 9] -Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: [ss_ticket_number#72 ASC NULLS FIRST, ss_item_sk#69 ASC NULLS FIRST], false, 0 +(82) Sort [codegen id : 9] +Input [13]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80] +Arguments: [ss_ticket_number#71 ASC NULLS FIRST, ss_item_sk#68 ASC NULLS FIRST], false, 0 -(84) Scan parquet default.store_returns -Output [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +(83) Scan parquet default.store_returns +Output [3]: [sr_item_sk#83, sr_ticket_number#84, sr_returned_date_sk#85] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 10] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +(84) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#83, sr_ticket_number#84, sr_returned_date_sk#85] -(86) Filter [codegen id : 10] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] -Condition : (isnotnull(sr_ticket_number#85) AND isnotnull(sr_item_sk#84)) +(85) Filter [codegen id : 10] +Input [3]: [sr_item_sk#83, sr_ticket_number#84, sr_returned_date_sk#85] +Condition : (isnotnull(sr_ticket_number#84) AND isnotnull(sr_item_sk#83)) -(87) Project [codegen id : 10] -Output [2]: [sr_item_sk#84, sr_ticket_number#85] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +(86) Project [codegen id : 10] +Output [2]: [sr_item_sk#83, sr_ticket_number#84] +Input [3]: [sr_item_sk#83, sr_ticket_number#84, sr_returned_date_sk#85] -(88) Exchange -Input [2]: [sr_item_sk#84, sr_ticket_number#85] -Arguments: hashpartitioning(sr_ticket_number#85, sr_item_sk#84, 5), ENSURE_REQUIREMENTS, [id=#87] +(87) Exchange +Input [2]: [sr_item_sk#83, sr_ticket_number#84] +Arguments: hashpartitioning(sr_ticket_number#84, sr_item_sk#83, 5), ENSURE_REQUIREMENTS, [id=#86] -(89) Sort [codegen id : 11] -Input [2]: [sr_item_sk#84, sr_ticket_number#85] -Arguments: [sr_ticket_number#85 ASC NULLS FIRST, sr_item_sk#84 ASC NULLS FIRST], false, 0 +(88) Sort [codegen id : 11] +Input [2]: [sr_item_sk#83, sr_ticket_number#84] +Arguments: [sr_ticket_number#84 ASC NULLS FIRST, sr_item_sk#83 ASC NULLS FIRST], false, 0 -(90) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#72, ss_item_sk#69] -Right keys [2]: [sr_ticket_number#85, sr_item_sk#84] +(89) SortMergeJoin [codegen id : 12] +Left keys [2]: [ss_ticket_number#71, ss_item_sk#68] +Right keys [2]: [sr_ticket_number#84, sr_item_sk#83] Join condition: None -(91) Project [codegen id : 12] -Output [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] -Input [15]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, sr_item_sk#84, sr_ticket_number#85] +(90) Project [codegen id : 12] +Output [11]: [ss_net_paid#72, s_store_name#52, s_state#54, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, c_first_name#64, c_last_name#65, ca_state#58] +Input [15]: [s_store_name#52, s_state#54, ca_state#58, c_first_name#64, c_last_name#65, ss_item_sk#68, ss_ticket_number#71, ss_net_paid#72, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, sr_item_sk#83, sr_ticket_number#84] -(92) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] -Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#73))] -Aggregate Attributes [1]: [sum#88] -Results [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +(91) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#72, s_store_name#52, s_state#54, i_current_price#76, i_size#77, i_color#78, i_units#79, i_manager_id#80, c_first_name#64, c_last_name#65, ca_state#58] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#52, ca_state#58, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#72))] +Aggregate Attributes [1]: [sum#87] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#52, ca_state#58, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#88] -(93) Exchange -Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] -Arguments: hashpartitioning(c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, 5), ENSURE_REQUIREMENTS, [id=#90] +(92) Exchange +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#52, ca_state#58, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#88] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#52, ca_state#58, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, 5), ENSURE_REQUIREMENTS, [id=#89] -(94) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] -Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] -Functions [1]: [sum(UnscaledValue(ss_net_paid#73))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#73))#91] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#73))#91,17,2) AS netpaid#40] +(93) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#52, ca_state#58, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77, sum#88] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#52, ca_state#58, s_state#54, i_color#78, i_current_price#76, i_manager_id#80, i_units#79, i_size#77] +Functions [1]: [sum(UnscaledValue(ss_net_paid#72))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#72))#90] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#72))#90,17,2) AS netpaid#40] -(95) HashAggregate [codegen id : 13] +(94) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#92, count#93] -Results [2]: [sum#94, count#95] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] -(96) Exchange -Input [2]: [sum#94, count#95] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] +(95) Exchange +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] -(97) HashAggregate [codegen id : 14] -Input [2]: [sum#94, count#95] +(96) HashAggregate [codegen id : 14] +Input [2]: [sum#93, count#94] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#97] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#97)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#98] +Aggregate Attributes [1]: [avg(netpaid#40)#96] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] 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 3feab0ac8cab..48e952e08028 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 @@ -3,158 +3,157 @@ WholeStageCodegen (12) InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (11) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (14) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (13) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (12) - 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_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #12 - WholeStageCodegen (8) - Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Filter [paid] + Subquery #1 + WholeStageCodegen (14) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #10 + WholeStageCodegen (13) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (12) + 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_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #12 + WholeStageCodegen (8) + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #13 + WholeStageCodegen (4) + Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] + BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (3) + 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 [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_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_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_state,ca_zip,ca_country] + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #17 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + WholeStageCodegen (11) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #18 + WholeStageCodegen (10) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (5) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #13 - WholeStageCodegen (4) - Project [s_store_name,s_state,ca_state,c_first_name,c_last_name,ss_item_sk,ss_ticket_number,ss_net_paid] - BroadcastHashJoin [s_store_sk,c_customer_sk,ss_store_sk,ss_customer_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (3) - 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 [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_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_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_state,ca_zip,ca_country] - Filter [c_customer_sk,c_current_addr_sk,c_birth_country] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (10) + 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_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_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_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #4 + WholeStageCodegen (5) + 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 [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 [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_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + 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.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #17 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + 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_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + Filter [i_color,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + WholeStageCodegen (8) + Sort [sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (11) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #18 - WholeStageCodegen (10) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum(netpaid),sum,isEmpty] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (10) - 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_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_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_ticket_number,ss_item_sk] - InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #4 - WholeStageCodegen (5) - 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 [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 [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_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_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_state,ca_zip,ca_country] - Filter [c_customer_sk,c_current_addr_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (7) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - WholeStageCodegen (8) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #9 - WholeStageCodegen (7) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] 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 2686a277825d..6cbe13661ad6 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 @@ -1,53 +1,52 @@ == Physical Plan == -* Sort (49) -+- Exchange (48) - +- * Project (47) - +- * Filter (46) - +- * HashAggregate (45) - +- Exchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (14) - : : : : +- * SortMergeJoin Inner (13) - : : : : :- * Sort (6) - : : : : : +- Exchange (5) - : : : : : +- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- * Sort (12) - : : : : +- Exchange (11) - : : : : +- * Project (10) - : : : : +- * Filter (9) - : : : : +- * ColumnarToRow (8) - : : : : +- Scan parquet default.store_returns (7) - : : : +- BroadcastExchange (19) - : : : +- * Project (18) - : : : +- * Filter (17) - : : : +- * ColumnarToRow (16) - : : : +- Scan parquet default.store (15) - : : +- BroadcastExchange (25) - : : +- * Filter (24) - : : +- * ColumnarToRow (23) - : : +- Scan parquet default.item (22) - : +- BroadcastExchange (31) - : +- * Filter (30) - : +- * ColumnarToRow (29) - : +- Scan parquet default.customer (28) - +- BroadcastExchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer_address (34) +* Sort (48) ++- Exchange (47) + +- * Filter (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * Project (4) + : : : : : +- * Filter (3) + : : : : : +- * ColumnarToRow (2) + : : : : : +- Scan parquet default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * Project (10) + : : : : +- * Filter (9) + : : : : +- * ColumnarToRow (8) + : : : : +- Scan parquet default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- * ColumnarToRow (16) + : : : +- Scan parquet default.store (15) + : : +- BroadcastExchange (25) + : : +- * Filter (24) + : : +- * ColumnarToRow (23) + : : +- Scan parquet default.item (22) + : +- BroadcastExchange (31) + : +- * Filter (30) + : +- * ColumnarToRow (29) + : +- Scan parquet default.customer (28) + +- BroadcastExchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer_address (34) (1) Scan parquet default.store_sales @@ -257,281 +256,277 @@ Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum#43, isEmpty#44 Keys [3]: [c_last_name#28, c_first_name#27, s_store_name#13] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#46] -Results [5]: [c_last_name#28, c_first_name#27, s_store_name#13, sum(netpaid#40)#46 AS paid#47, sum(netpaid#40)#46 AS sum(netpaid#40)#48] +Results [4]: [c_last_name#28, c_first_name#27, s_store_name#13, sum(netpaid#40)#46 AS paid#47] (46) Filter [codegen id : 11] -Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47, sum(netpaid#40)#48] -Condition : (isnotnull(sum(netpaid#40)#48) AND (cast(sum(netpaid#40)#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(47) Project [codegen id : 11] -Output [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] -Input [5]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47, sum(netpaid#40)#48] +Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] +Condition : (isnotnull(paid#47) AND (cast(paid#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) -(48) Exchange +(47) Exchange Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] -Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#51] +Arguments: rangepartitioning(c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#50] -(49) Sort [codegen id : 12] +(48) Sort [codegen id : 12] Input [4]: [c_last_name#28, c_first_name#27, s_store_name#13, paid#47] Arguments: [c_last_name#28 ASC NULLS FIRST, c_first_name#27 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (94) -+- Exchange (93) - +- * HashAggregate (92) - +- * HashAggregate (91) - +- Exchange (90) - +- * HashAggregate (89) - +- * Project (88) - +- * BroadcastHashJoin Inner BuildRight (87) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (76) - : : +- * BroadcastHashJoin Inner BuildRight (75) - : : :- * Project (70) - : : : +- * BroadcastHashJoin Inner BuildRight (69) - : : : :- * Project (63) - : : : : +- * SortMergeJoin Inner (62) - : : : : :- * Sort (55) - : : : : : +- Exchange (54) - : : : : : +- * Project (53) - : : : : : +- * Filter (52) - : : : : : +- * ColumnarToRow (51) - : : : : : +- Scan parquet default.store_sales (50) - : : : : +- * Sort (61) - : : : : +- Exchange (60) - : : : : +- * Project (59) - : : : : +- * Filter (58) - : : : : +- * ColumnarToRow (57) - : : : : +- Scan parquet default.store_returns (56) - : : : +- BroadcastExchange (68) - : : : +- * Project (67) - : : : +- * Filter (66) - : : : +- * ColumnarToRow (65) - : : : +- Scan parquet default.store (64) - : : +- BroadcastExchange (74) - : : +- * Filter (73) - : : +- * ColumnarToRow (72) - : : +- Scan parquet default.item (71) - : +- BroadcastExchange (80) - : +- * Filter (79) - : +- * ColumnarToRow (78) - : +- Scan parquet default.customer (77) - +- BroadcastExchange (86) - +- * Filter (85) - +- * ColumnarToRow (84) - +- Scan parquet default.customer_address (83) - - -(50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (93) ++- Exchange (92) + +- * HashAggregate (91) + +- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- * Project (87) + +- * BroadcastHashJoin Inner BuildRight (86) + :- * Project (81) + : +- * BroadcastHashJoin Inner BuildRight (80) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * Project (69) + : : : +- * BroadcastHashJoin Inner BuildRight (68) + : : : :- * Project (62) + : : : : +- * SortMergeJoin Inner (61) + : : : : :- * Sort (54) + : : : : : +- Exchange (53) + : : : : : +- * Project (52) + : : : : : +- * Filter (51) + : : : : : +- * ColumnarToRow (50) + : : : : : +- Scan parquet default.store_sales (49) + : : : : +- * Sort (60) + : : : : +- Exchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store_returns (55) + : : : +- BroadcastExchange (67) + : : : +- * Project (66) + : : : +- * Filter (65) + : : : +- * ColumnarToRow (64) + : : : +- Scan parquet default.store (63) + : : +- BroadcastExchange (73) + : : +- * Filter (72) + : : +- * ColumnarToRow (71) + : : +- Scan parquet default.item (70) + : +- BroadcastExchange (79) + : +- * Filter (78) + : +- * ColumnarToRow (77) + : +- Scan parquet default.customer (76) + +- BroadcastExchange (85) + +- * Filter (84) + +- * ColumnarToRow (83) + +- Scan parquet default.customer_address (82) + + +(49) Scan parquet default.store_sales +Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(51) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +(50) ColumnarToRow [codegen id : 1] +Input [6]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, ss_sold_date_sk#56] -(52) Filter [codegen id : 1] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] -Condition : (((isnotnull(ss_ticket_number#55) AND isnotnull(ss_item_sk#52)) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_customer_sk#53)) +(51) Filter [codegen id : 1] +Input [6]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : (((isnotnull(ss_ticket_number#54) AND isnotnull(ss_item_sk#51)) AND isnotnull(ss_store_sk#53)) AND isnotnull(ss_customer_sk#52)) -(53) Project [codegen id : 1] -Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +(52) Project [codegen id : 1] +Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] +Input [6]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, ss_sold_date_sk#56] -(54) Exchange -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: hashpartitioning(ss_ticket_number#55, ss_item_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] +(53) Exchange +Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] +Arguments: hashpartitioning(ss_ticket_number#54, ss_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#57] -(55) Sort [codegen id : 2] -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: [ss_ticket_number#55 ASC NULLS FIRST, ss_item_sk#52 ASC NULLS FIRST], false, 0 +(54) Sort [codegen id : 2] +Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] +Arguments: [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST], false, 0 -(56) Scan parquet default.store_returns -Output [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +(55) Scan parquet default.store_returns +Output [3]: [sr_item_sk#58, sr_ticket_number#59, sr_returned_date_sk#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(57) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +(56) ColumnarToRow [codegen id : 3] +Input [3]: [sr_item_sk#58, sr_ticket_number#59, sr_returned_date_sk#60] -(58) Filter [codegen id : 3] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] -Condition : (isnotnull(sr_ticket_number#60) AND isnotnull(sr_item_sk#59)) +(57) Filter [codegen id : 3] +Input [3]: [sr_item_sk#58, sr_ticket_number#59, sr_returned_date_sk#60] +Condition : (isnotnull(sr_ticket_number#59) AND isnotnull(sr_item_sk#58)) -(59) Project [codegen id : 3] -Output [2]: [sr_item_sk#59, sr_ticket_number#60] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +(58) Project [codegen id : 3] +Output [2]: [sr_item_sk#58, sr_ticket_number#59] +Input [3]: [sr_item_sk#58, sr_ticket_number#59, sr_returned_date_sk#60] -(60) Exchange -Input [2]: [sr_item_sk#59, sr_ticket_number#60] -Arguments: hashpartitioning(sr_ticket_number#60, sr_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#62] +(59) Exchange +Input [2]: [sr_item_sk#58, sr_ticket_number#59] +Arguments: hashpartitioning(sr_ticket_number#59, sr_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#61] -(61) Sort [codegen id : 4] -Input [2]: [sr_item_sk#59, sr_ticket_number#60] -Arguments: [sr_ticket_number#60 ASC NULLS FIRST, sr_item_sk#59 ASC NULLS FIRST], false, 0 +(60) Sort [codegen id : 4] +Input [2]: [sr_item_sk#58, sr_ticket_number#59] +Arguments: [sr_ticket_number#59 ASC NULLS FIRST, sr_item_sk#58 ASC NULLS FIRST], false, 0 -(62) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#55, ss_item_sk#52] -Right keys [2]: [sr_ticket_number#60, sr_item_sk#59] +(61) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#54, ss_item_sk#51] +Right keys [2]: [sr_ticket_number#59, sr_item_sk#58] Join condition: None -(63) Project [codegen id : 9] -Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] -Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#59, sr_ticket_number#60] +(62) Project [codegen id : 9] +Output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] +Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#58, sr_ticket_number#59] -(64) Scan parquet default.store -Output [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +(63) Scan parquet default.store +Output [5]: [s_store_sk#62, s_store_name#63, s_market_id#64, s_state#65, s_zip#66] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +(64) ColumnarToRow [codegen id : 5] +Input [5]: [s_store_sk#62, s_store_name#63, s_market_id#64, s_state#65, s_zip#66] -(66) Filter [codegen id : 5] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] -Condition : (((isnotnull(s_market_id#65) AND (s_market_id#65 = 8)) AND isnotnull(s_store_sk#63)) AND isnotnull(s_zip#67)) +(65) Filter [codegen id : 5] +Input [5]: [s_store_sk#62, s_store_name#63, s_market_id#64, s_state#65, s_zip#66] +Condition : (((isnotnull(s_market_id#64) AND (s_market_id#64 = 8)) AND isnotnull(s_store_sk#62)) AND isnotnull(s_zip#66)) -(67) Project [codegen id : 5] -Output [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +(66) Project [codegen id : 5] +Output [4]: [s_store_sk#62, s_store_name#63, s_state#65, s_zip#66] +Input [5]: [s_store_sk#62, s_store_name#63, s_market_id#64, s_state#65, s_zip#66] -(68) BroadcastExchange -Input [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] +(67) BroadcastExchange +Input [4]: [s_store_sk#62, s_store_name#63, s_state#65, s_zip#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#67] -(69) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#63] +(68) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#53] +Right keys [1]: [s_store_sk#62] Join condition: None -(70) Project [codegen id : 9] -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67] -Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +(69) Project [codegen id : 9] +Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#63, s_state#65, s_zip#66] +Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#62, s_store_name#63, s_state#65, s_zip#66] -(71) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +(70) Scan parquet default.item +Output [6]: [i_item_sk#68, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(72) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +(71) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#68, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73] -(73) Filter [codegen id : 6] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +(72) Filter [codegen id : 6] +Input [6]: [i_item_sk#68, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73] +Condition : isnotnull(i_item_sk#68) -(74) BroadcastExchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +(73) BroadcastExchange +Input [6]: [i_item_sk#68, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] -(75) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#69] +(74) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#51] +Right keys [1]: [i_item_sk#68] Join condition: None -(76) Project [codegen id : 9] -Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +(75) Project [codegen id : 9] +Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#63, s_state#65, s_zip#66, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73] +Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#63, s_state#65, s_zip#66, i_item_sk#68, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73] -(77) Scan parquet default.customer -Output [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +(76) Scan parquet default.customer +Output [5]: [c_customer_sk#75, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(78) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +(77) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#75, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79] -(79) Filter [codegen id : 7] -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Condition : ((isnotnull(c_customer_sk#76) AND isnotnull(c_current_addr_sk#77)) AND isnotnull(c_birth_country#80)) +(78) Filter [codegen id : 7] +Input [5]: [c_customer_sk#75, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79] +Condition : ((isnotnull(c_customer_sk#75) AND isnotnull(c_current_addr_sk#76)) AND isnotnull(c_birth_country#79)) -(80) BroadcastExchange -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] +(79) BroadcastExchange +Input [5]: [c_customer_sk#75, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] -(81) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#53] -Right keys [1]: [c_customer_sk#76] +(80) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#52] +Right keys [1]: [c_customer_sk#75] Join condition: None -(82) Project [codegen id : 9] -Output [13]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +(81) Project [codegen id : 9] +Output [13]: [ss_net_paid#55, s_store_name#63, s_state#65, s_zip#66, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79] +Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#63, s_state#65, s_zip#66, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73, c_customer_sk#75, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79] -(83) Scan parquet default.customer_address -Output [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +(82) Scan parquet default.customer_address +Output [4]: [ca_address_sk#81, ca_state#82, ca_zip#83, ca_country#84] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +(83) ColumnarToRow [codegen id : 8] +Input [4]: [ca_address_sk#81, ca_state#82, ca_zip#83, ca_country#84] -(85) Filter [codegen id : 8] -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] -Condition : ((isnotnull(ca_address_sk#82) AND isnotnull(ca_country#85)) AND isnotnull(ca_zip#84)) +(84) Filter [codegen id : 8] +Input [4]: [ca_address_sk#81, ca_state#82, ca_zip#83, ca_country#84] +Condition : ((isnotnull(ca_address_sk#81) AND isnotnull(ca_country#84)) AND isnotnull(ca_zip#83)) -(86) BroadcastExchange -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#86] +(85) BroadcastExchange +Input [4]: [ca_address_sk#81, ca_state#82, ca_zip#83, ca_country#84] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#85] -(87) BroadcastHashJoin [codegen id : 9] -Left keys [3]: [c_current_addr_sk#77, c_birth_country#80, s_zip#67] -Right keys [3]: [ca_address_sk#82, upper(ca_country#85), ca_zip#84] +(86) BroadcastHashJoin [codegen id : 9] +Left keys [3]: [c_current_addr_sk#76, c_birth_country#79, s_zip#66] +Right keys [3]: [ca_address_sk#81, upper(ca_country#84), ca_zip#83] Join condition: None -(88) Project [codegen id : 9] -Output [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] -Input [17]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80, ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +(87) Project [codegen id : 9] +Output [11]: [ss_net_paid#55, s_store_name#63, s_state#65, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73, c_first_name#77, c_last_name#78, ca_state#82] +Input [17]: [ss_net_paid#55, s_store_name#63, s_state#65, s_zip#66, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73, c_current_addr_sk#76, c_first_name#77, c_last_name#78, c_birth_country#79, ca_address_sk#81, ca_state#82, ca_zip#83, ca_country#84] -(89) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] -Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum#87] -Results [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +(88) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#55, s_store_name#63, s_state#65, i_current_price#69, i_size#70, i_color#71, i_units#72, i_manager_id#73, c_first_name#77, c_last_name#78, ca_state#82] +Keys [10]: [c_last_name#78, c_first_name#77, s_store_name#63, ca_state#82, s_state#65, i_color#71, i_current_price#69, i_manager_id#73, i_units#72, i_size#70] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#78, c_first_name#77, s_store_name#63, ca_state#82, s_state#65, i_color#71, i_current_price#69, i_manager_id#73, i_units#72, i_size#70, sum#87] -(90) Exchange -Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] -Arguments: hashpartitioning(c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#89] +(89) Exchange +Input [11]: [c_last_name#78, c_first_name#77, s_store_name#63, ca_state#82, s_state#65, i_color#71, i_current_price#69, i_manager_id#73, i_units#72, i_size#70, sum#87] +Arguments: hashpartitioning(c_last_name#78, c_first_name#77, s_store_name#63, ca_state#82, s_state#65, i_color#71, i_current_price#69, i_manager_id#73, i_units#72, i_size#70, 5), ENSURE_REQUIREMENTS, [id=#88] -(91) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] -Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#90] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#90,17,2) AS netpaid#40] +(90) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#78, c_first_name#77, s_store_name#63, ca_state#82, s_state#65, i_color#71, i_current_price#69, i_manager_id#73, i_units#72, i_size#70, sum#87] +Keys [10]: [c_last_name#78, c_first_name#77, s_store_name#63, ca_state#82, s_state#65, i_color#71, i_current_price#69, i_manager_id#73, i_units#72, i_size#70] +Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#89,17,2) AS netpaid#40] -(92) HashAggregate [codegen id : 10] +(91) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(93) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +(92) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(94) HashAggregate [codegen id : 11] -Input [2]: [sum#93, count#94] +(93) HashAggregate [codegen id : 11] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#96] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] +Aggregate Attributes [1]: [avg(netpaid#40)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 6309ec999ce7..6c43f9a35d3d 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 @@ -3,149 +3,148 @@ WholeStageCodegen (12) InputAdapter Exchange [c_last_name,c_first_name,s_store_name] #1 WholeStageCodegen (11) - Project [c_last_name,c_first_name,s_store_name,paid] - Filter [sum(netpaid)] - Subquery #1 - WholeStageCodegen (11) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - WholeStageCodegen (10) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - InputAdapter - Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (9) - 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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #10 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (9) + 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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #12 + WholeStageCodegen (1) + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #13 + WholeStageCodegen (3) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (5) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #12 - WholeStageCodegen (1) - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #13 - WholeStageCodegen (3) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (7) + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] + ColumnarToRow InputAdapter - BroadcastExchange #14 - WholeStageCodegen (5) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (8) + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow InputAdapter - BroadcastExchange #15 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (7) - Filter [c_customer_sk,c_current_addr_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (8) - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow + 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,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (10) + 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_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_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] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - 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_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (10) - 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_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_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] - SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - WholeStageCodegen (2) - Sort [ss_ticket_number,ss_item_sk] + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #4 + WholeStageCodegen (1) + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #5 + WholeStageCodegen (3) + Project [sr_item_sk,sr_ticket_number] + Filter [sr_ticket_number,sr_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [s_store_sk,s_store_name,s_state,s_zip] + Filter [s_market_id,s_store_sk,s_zip] + ColumnarToRow InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #4 - WholeStageCodegen (1) - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - Filter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Filter [i_color,i_item_sk] + ColumnarToRow InputAdapter - WholeStageCodegen (4) - Sort [sr_ticket_number,sr_item_sk] - InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #5 - WholeStageCodegen (3) - Project [sr_item_sk,sr_ticket_number] - Filter [sr_ticket_number,sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + Filter [c_customer_sk,c_current_addr_sk,c_birth_country] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - 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_store_sk,s_store_name,s_market_id,s_state,s_zip] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Filter [ca_address_sk,ca_country,ca_zip] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Filter [i_color,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - Filter [c_customer_sk,c_current_addr_sk,c_birth_country] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Filter [ca_address_sk,ca_country,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + 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/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index 3f2b62a7b8fa..8180e3b63d2b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -1,54 +1,53 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * Filter (48) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * SortMergeJoin Inner (43) - :- * Sort (28) - : +- Exchange (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.store_sales (1) - : : +- BroadcastExchange (17) - : : +- * Project (16) - : : +- * Filter (15) - : : +- * BroadcastHashJoin LeftOuter BuildRight (14) - : : :- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (13) - : : +- * HashAggregate (12) - : : +- Exchange (11) - : : +- * HashAggregate (10) - : : +- * Filter (9) - : : +- * ColumnarToRow (8) - : : +- Scan parquet default.item (7) - : +- BroadcastExchange (24) - : +- * Project (23) - : +- * Filter (22) - : +- * ColumnarToRow (21) - : +- Scan parquet default.date_dim (20) - +- * Sort (42) - +- Exchange (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Filter (31) - : +- * ColumnarToRow (30) - : +- Scan parquet default.customer_address (29) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.customer (34) +TakeOrderedAndProject (49) ++- * Filter (48) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * SortMergeJoin Inner (43) + :- * Sort (28) + : +- Exchange (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.store_sales (1) + : : +- BroadcastExchange (17) + : : +- * Project (16) + : : +- * Filter (15) + : : +- * BroadcastHashJoin LeftOuter BuildRight (14) + : : :- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.item (4) + : : +- BroadcastExchange (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Filter (9) + : : +- * ColumnarToRow (8) + : : +- Scan parquet default.item (7) + : +- BroadcastExchange (24) + : +- * Project (23) + : +- * Filter (22) + : +- * ColumnarToRow (21) + : +- Scan parquet default.date_dim (20) + +- * Sort (42) + +- Exchange (41) + +- * Project (40) + +- * SortMergeJoin Inner (39) + :- * Sort (33) + : +- Exchange (32) + : +- * Filter (31) + : +- * ColumnarToRow (30) + : +- Scan parquet default.customer_address (29) + +- * Sort (38) + +- Exchange (37) + +- * Filter (36) + +- * ColumnarToRow (35) + +- Scan parquet default.customer (34) (1) Scan parquet default.store_sales @@ -267,73 +266,69 @@ Input [2]: [ca_state#26, count#33] Keys [1]: [ca_state#26] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#35] -Results [4]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38, ca_state#26] +Results [3]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, ca_state#26] (48) Filter [codegen id : 14] -Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] -Condition : (count(1)#38 >= 10) - -(49) Project [codegen id : 14] -Output [3]: [state#36, cnt#37, ca_state#26] -Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] +Input [3]: [state#36, cnt#37, ca_state#26] +Condition : (cnt#37 >= 10) -(50) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [3]: [state#36, cnt#37, ca_state#26] Arguments: 100, [cnt#37 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -ReusedExchange (51) +ReusedExchange (50) -(51) ReusedExchange [Reuses operator id: 24] +(50) ReusedExchange [Reuses operator id: 24] Output [1]: [d_date_sk#19] Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] -* HashAggregate (58) -+- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * Filter (54) - +- * ColumnarToRow (53) - +- Scan parquet default.date_dim (52) +* HashAggregate (57) ++- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.date_dim (51) -(52) Scan parquet default.date_dim -Output [3]: [d_month_seq#39, d_year#40, d_moy#41] +(51) Scan parquet default.date_dim +Output [3]: [d_month_seq#38, d_year#39, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +(52) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#38, d_year#39, d_moy#40] -(54) Filter [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] -Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) +(53) Filter [codegen id : 1] +Input [3]: [d_month_seq#38, d_year#39, d_moy#40] +Condition : (((isnotnull(d_year#39) AND isnotnull(d_moy#40)) AND (d_year#39 = 2000)) AND (d_moy#40 = 1)) -(55) Project [codegen id : 1] -Output [1]: [d_month_seq#39] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +(54) Project [codegen id : 1] +Output [1]: [d_month_seq#38] +Input [3]: [d_month_seq#38, d_year#39, d_moy#40] -(56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +(55) HashAggregate [codegen id : 1] +Input [1]: [d_month_seq#38] +Keys [1]: [d_month_seq#38] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#38] -(57) Exchange -Input [1]: [d_month_seq#39] -Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] +(56) Exchange +Input [1]: [d_month_seq#38] +Arguments: hashpartitioning(d_month_seq#38, 5), ENSURE_REQUIREMENTS, [id=#41] -(58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +(57) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#38] +Keys [1]: [d_month_seq#38] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#38] 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 f3badf6efe6b..36371f422391 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,97 +1,96 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (14) - Project [state,cnt,ca_state] - Filter [count(1)] - 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 [ss_customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #2 - WholeStageCodegen (5) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_customer_sk,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [i_item_sk] - 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_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] - InputAdapter - Exchange [i_category] #6 - WholeStageCodegen (1) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_current_price,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [d_month_seq] - InputAdapter - Exchange [d_month_seq] #7 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - 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] - InputAdapter - WholeStageCodegen (12) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (11) - Project [ca_state,c_customer_sk] - SortMergeJoin [ca_address_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (8) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #9 - WholeStageCodegen (7) - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - WholeStageCodegen (10) - Sort [c_current_addr_sk] + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + InputAdapter + Exchange [ca_state] #1 + WholeStageCodegen (13) + HashAggregate [ca_state] [count,count] + Project [ca_state] + SortMergeJoin [ss_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #2 + WholeStageCodegen (5) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_customer_sk,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_customer_sk,ss_item_sk] + ColumnarToRow InputAdapter - Exchange [c_current_addr_sk] #10 - WholeStageCodegen (9) - Filter [c_current_addr_sk,c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Project [i_item_sk] + 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_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #6 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_current_price,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (4) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #7 + WholeStageCodegen (1) + HashAggregate [d_month_seq] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + 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] + InputAdapter + WholeStageCodegen (12) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (11) + Project [ca_state,c_customer_sk] + SortMergeJoin [ca_address_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (8) + Sort [ca_address_sk] + InputAdapter + Exchange [ca_address_sk] #9 + WholeStageCodegen (7) + Filter [ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + WholeStageCodegen (10) + Sort [c_current_addr_sk] + InputAdapter + Exchange [c_current_addr_sk] #10 + WholeStageCodegen (9) + Filter [c_current_addr_sk,c_customer_sk] + ColumnarToRow + InputAdapter + 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/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index b37db85388e0..065d960c92f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -1,48 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * Filter (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.store_sales (10) - : +- BroadcastExchange (20) - : +- * Project (19) - : +- * Filter (18) - : +- * ColumnarToRow (17) - : +- Scan parquet default.date_dim (16) - +- BroadcastExchange (36) - +- * Project (35) - +- * Filter (34) - +- * BroadcastHashJoin LeftOuter BuildRight (33) - :- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.item (23) - +- BroadcastExchange (32) - +- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Filter (28) - +- * ColumnarToRow (27) - +- Scan parquet default.item (26) +TakeOrderedAndProject (43) ++- * Filter (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.customer (4) + : : +- BroadcastExchange (13) + : : +- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet default.store_sales (10) + : +- BroadcastExchange (20) + : +- * Project (19) + : +- * Filter (18) + : +- * ColumnarToRow (17) + : +- Scan parquet default.date_dim (16) + +- BroadcastExchange (36) + +- * Project (35) + +- * Filter (34) + +- * BroadcastHashJoin LeftOuter BuildRight (33) + :- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.item (23) + +- BroadcastExchange (32) + +- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Filter (28) + +- * ColumnarToRow (27) + +- Scan parquet default.item (26) (1) Scan parquet default.customer_address @@ -237,73 +236,69 @@ Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#33] -Results [4]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36, ca_state#2] +Results [3]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, ca_state#2] (42) Filter [codegen id : 8] -Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] -Condition : (count(1)#36 >= 10) - -(43) Project [codegen id : 8] -Output [3]: [state#34, cnt#35, ca_state#2] -Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] +Input [3]: [state#34, cnt#35, ca_state#2] +Condition : (cnt#35 >= 10) -(44) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [3]: [state#34, cnt#35, ca_state#2] Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -ReusedExchange (45) +ReusedExchange (44) -(45) ReusedExchange [Reuses operator id: 20] +(44) ReusedExchange [Reuses operator id: 20] Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* HashAggregate (52) -+- Exchange (51) - +- * HashAggregate (50) - +- * Project (49) - +- * Filter (48) - +- * ColumnarToRow (47) - +- Scan parquet default.date_dim (46) +* HashAggregate (51) ++- Exchange (50) + +- * HashAggregate (49) + +- * Project (48) + +- * Filter (47) + +- * ColumnarToRow (46) + +- Scan parquet default.date_dim (45) -(46) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#38, d_moy#39] +(45) Scan parquet default.date_dim +Output [3]: [d_month_seq#36, d_year#37, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +(46) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#36, d_year#37, d_moy#38] -(48) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +(47) Filter [codegen id : 1] +Input [3]: [d_month_seq#36, d_year#37, d_moy#38] +Condition : (((isnotnull(d_year#37) AND isnotnull(d_moy#38)) AND (d_year#37 = 2000)) AND (d_moy#38 = 1)) -(49) Project [codegen id : 1] -Output [1]: [d_month_seq#37] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +(48) Project [codegen id : 1] +Output [1]: [d_month_seq#36] +Input [3]: [d_month_seq#36, d_year#37, d_moy#38] -(50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +(49) HashAggregate [codegen id : 1] +Input [1]: [d_month_seq#36] +Keys [1]: [d_month_seq#36] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#36] -(51) Exchange -Input [1]: [d_month_seq#37] -Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] +(50) Exchange +Input [1]: [d_month_seq#36] +Arguments: hashpartitioning(d_month_seq#36, 5), ENSURE_REQUIREMENTS, [id=#39] -(52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#36] +Keys [1]: [d_month_seq#36] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#36] 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 4ba09283e73c..5514db3a4442 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,79 +1,78 @@ TakeOrderedAndProject [cnt,ca_state,state] WholeStageCodegen (8) - Project [state,cnt,ca_state] - Filter [count(1)] - 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 [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - Filter [ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [c_current_addr_sk,c_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [d_month_seq] + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + InputAdapter + Exchange [ca_state] #1 + WholeStageCodegen (7) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + Filter [ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + Filter [c_current_addr_sk,c_customer_sk] + ColumnarToRow InputAdapter - Exchange [d_month_seq] #5 - WholeStageCodegen (1) - HashAggregate [d_month_seq] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [i_item_sk] - Filter [i_current_price,avg(i_current_price)] - BroadcastHashJoin [i_category,i_category] - Filter [i_current_price,i_item_sk] + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Filter [ss_customer_sk,ss_item_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] - InputAdapter - Exchange [i_category] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_current_price] [sum,count,sum,count] - Filter [i_category] + Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + HashAggregate [d_month_seq] + Project [d_month_seq] + Filter [d_year,d_moy] ColumnarToRow InputAdapter - Scan parquet default.item [i_current_price,i_category] + 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] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [i_item_sk] + 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_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + Filter [i_category] + ColumnarToRow + InputAdapter + 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 c828857edc4b..b4e31bf3b8ad 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 @@ -351,15 +351,15 @@ Input [4]: [cs_item_sk#19, sum#34, sum#35, isEmpty#36] Keys [1]: [cs_item_sk#19] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#39] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#38,17,2) AS sum(cs_ext_list_price#21)#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41] +Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41] (31) Filter [codegen id : 10] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41] -Condition : (isnotnull(sum(cs_ext_list_price#21)#40) AND (cast(sum(cs_ext_list_price#21)#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#19, sale#40, refund#41] +Condition : (isnotnull(sale#40) AND (cast(sale#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#41)), DecimalType(21,2), true))) (32) Project [codegen id : 10] Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41] +Input [3]: [cs_item_sk#19, sale#40, refund#41] (33) BroadcastExchange Input [1]: [cs_item_sk#19] 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 22228c7657d2..0cb1e60ea2ac 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 @@ -107,8 +107,8 @@ WholeStageCodegen (88) BroadcastExchange #12 WholeStageCodegen (10) Project [cs_item_sk] - 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] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #13 WholeStageCodegen (9) 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 367301da51bf..0586e176ba74 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 @@ -321,15 +321,15 @@ Input [4]: [cs_item_sk#19, sum#34, sum#35, isEmpty#36] Keys [1]: [cs_item_sk#19] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#39] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#38,17,2) AS sum(cs_ext_list_price#21)#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41] +Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41] (30) Filter [codegen id : 9] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41] -Condition : (isnotnull(sum(cs_ext_list_price#21)#40) AND (cast(sum(cs_ext_list_price#21)#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#19, sale#40, refund#41] +Condition : (isnotnull(sale#40) AND (cast(sale#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#41)), DecimalType(21,2), true))) (31) Project [codegen id : 9] Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#41] +Input [3]: [cs_item_sk#19, sale#40, refund#41] (32) Sort [codegen id : 9] Input [1]: [cs_item_sk#19] @@ -762,15 +762,15 @@ Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] Keys [1]: [cs_item_sk#140] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] -Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS refund#41] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] -Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sale#40, refund#41] +Condition : (isnotnull(sale#40) AND (cast(sale#40 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#41)), DecimalType(21,2), true))) (130) Project [codegen id : 35] Output [1]: [cs_item_sk#140] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Input [3]: [cs_item_sk#140, sale#40, refund#41] (131) Sort [codegen id : 35] Input [1]: [cs_item_sk#140] @@ -786,238 +786,238 @@ Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#1 Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#150, d_year#151] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] +Input [2]: [d_date_sk#150, d_year#151] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] -Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) +Input [2]: [d_date_sk#150, d_year#151] +Condition : ((isnotnull(d_year#151) AND (d_year#151 = 2000)) AND isnotnull(d_date_sk#150)) (137) BroadcastExchange -Input [2]: [d_date_sk#152, d_year#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] +Input [2]: [d_date_sk#150, d_year#151] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#152] (138) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_sold_date_sk#133] -Right keys [1]: [d_date_sk#152] +Right keys [1]: [d_date_sk#150] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] -Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#150, d_year#151] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] +Output [3]: [s_store_sk#153, s_store_name#154, s_zip#155] (141) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_store_sk#127] -Right keys [1]: [s_store_sk#155] +Right keys [1]: [s_store_sk#153] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_sk#153, s_store_name#154, s_zip#155] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [6]: [c_customer_sk#156, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, c_first_shipto_date_sk#160, c_first_sales_date_sk#161] (144) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_customer_sk#123] -Right keys [1]: [c_customer_sk#158] +Right keys [1]: [c_customer_sk#156] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] -Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, c_first_shipto_date_sk#160, c_first_sales_date_sk#161] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_customer_sk#156, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, c_first_shipto_date_sk#160, c_first_sales_date_sk#161] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#164, d_year#165] +Output [2]: [d_date_sk#162, d_year#163] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#163] -Right keys [1]: [d_date_sk#164] +Left keys [1]: [c_first_sales_date_sk#161] +Right keys [1]: [d_date_sk#162] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, c_first_shipto_date_sk#160, d_year#163] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, c_first_shipto_date_sk#160, c_first_sales_date_sk#161, d_date_sk#162, d_year#163] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#166, d_year#167] +Output [2]: [d_date_sk#164, d_year#165] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#162] -Right keys [1]: [d_date_sk#166] +Left keys [1]: [c_first_shipto_date_sk#160] +Right keys [1]: [d_date_sk#164] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, c_first_shipto_date_sk#160, d_year#163, d_date_sk#164, d_year#165] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#168, cd_marital_status#169] +Output [2]: [cd_demo_sk#166, cd_marital_status#167] (153) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_cdemo_sk#124] -Right keys [1]: [cd_demo_sk#168] +Right keys [1]: [cd_demo_sk#166] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, cd_marital_status#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, cd_demo_sk#166, cd_marital_status#167] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#170, cd_marital_status#171] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#159] -Right keys [1]: [cd_demo_sk#170] -Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) +Left keys [1]: [c_current_cdemo_sk#157] +Right keys [1]: [cd_demo_sk#168] +Join condition: NOT (cd_marital_status#167 = cd_marital_status#169) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_cdemo_sk#157, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, cd_marital_status#167, cd_demo_sk#168, cd_marital_status#169] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#172] +Output [1]: [p_promo_sk#170] (159) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_promo_sk#128] -Right keys [1]: [p_promo_sk#172] +Right keys [1]: [p_promo_sk#170] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, p_promo_sk#170] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] +Output [2]: [hd_demo_sk#171, hd_income_band_sk#172] (162) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_hdemo_sk#125] -Right keys [1]: [hd_demo_sk#173] +Right keys [1]: [hd_demo_sk#171] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, hd_income_band_sk#172] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, hd_demo_sk#171, hd_income_band_sk#172] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#160] -Right keys [1]: [hd_demo_sk#175] +Left keys [1]: [c_current_hdemo_sk#158] +Right keys [1]: [hd_demo_sk#173] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] -Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_addr_sk#159, d_year#163, d_year#165, hd_income_band_sk#172, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_hdemo_sk#158, c_current_addr_sk#159, d_year#163, d_year#165, hd_income_band_sk#172, hd_demo_sk#173, hd_income_band_sk#174] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (168) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_addr_sk#126] -Right keys [1]: [ca_address_sk#177] +Right keys [1]: [ca_address_sk#175] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] -Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_addr_sk#159, d_year#163, d_year#165, hd_income_band_sk#172, hd_income_band_sk#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_addr_sk#159, d_year#163, d_year#165, hd_income_band_sk#172, hd_income_band_sk#174, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [5]: [ca_address_sk#180, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#182] +Left keys [1]: [c_current_addr_sk#159] +Right keys [1]: [ca_address_sk#180] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, d_year#163, d_year#165, hd_income_band_sk#172, hd_income_band_sk#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, c_current_addr_sk#159, d_year#163, d_year#165, hd_income_band_sk#172, hd_income_band_sk#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#180, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#185] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#174] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#172] +Right keys [1]: [ib_income_band_sk#185] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, d_year#163, d_year#165, hd_income_band_sk#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, d_year#163, d_year#165, hd_income_band_sk#172, hd_income_band_sk#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, ib_income_band_sk#185] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#188] +Output [1]: [ib_income_band_sk#186] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#176] -Right keys [1]: [ib_income_band_sk#188] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, d_year#163, d_year#165, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, d_year#163, d_year#165, hd_income_band_sk#174, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, ib_income_band_sk#186] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#189, i_product_name#190] +Output [2]: [i_item_sk#187, i_product_name#188] (180) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_item_sk#122] -Right keys [1]: [i_item_sk#189] +Right keys [1]: [i_item_sk#187] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, d_year#163, d_year#165, s_store_name#154, s_zip#155, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, i_item_sk#187, i_product_name#188] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, s_store_name#154, s_zip#155, d_year#163, d_year#165, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, i_item_sk#187, i_product_name#188] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#151, d_year#163, d_year#165, s_store_name#154, s_zip#155, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, i_item_sk#187, i_product_name#188] +Keys [15]: [i_product_name#188, i_item_sk#187, s_store_name#154, s_zip#155, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, d_year#151, d_year#163, d_year#165] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] -Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Aggregate Attributes [4]: [count#189, sum#190, sum#191, sum#192] +Results [19]: [i_product_name#188, i_item_sk#187, s_store_name#154, s_zip#155, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, d_year#151, d_year#163, d_year#165, count#193, sum#194, sum#195, sum#196] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Input [19]: [i_product_name#188, i_item_sk#187, s_store_name#154, s_zip#155, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, d_year#151, d_year#163, d_year#165, count#193, sum#194, sum#195, sum#196] +Keys [15]: [i_product_name#188, i_item_sk#187, s_store_name#154, s_zip#155, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#181, ca_street_name#182, ca_city#183, ca_zip#184, d_year#151, d_year#163, d_year#165] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] -Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] +Aggregate Attributes [4]: [count(1)#197, sum(UnscaledValue(ss_wholesale_cost#130))#198, sum(UnscaledValue(ss_list_price#131))#199, sum(UnscaledValue(ss_coupon_amt#132))#200] +Results [8]: [i_item_sk#187 AS item_sk#201, s_store_name#154 AS store_name#202, s_zip#155 AS store_zip#203, d_year#151 AS syear#204, count(1)#197 AS cnt#205, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#198,17,2) AS s1#206, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#199,17,2) AS s2#207, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#200,17,2) AS s3#208] (184) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#201, store_name#202, store_zip#203, syear#204, cnt#205, s1#206, s2#207, s3#208] +Arguments: hashpartitioning(item_sk#201, store_name#202, store_zip#203, 5), ENSURE_REQUIREMENTS, [id=#209] (185) Sort [codegen id : 52] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#201, store_name#202, store_zip#203, syear#204, cnt#205, s1#206, s2#207, s3#208] +Arguments: [item_sk#201 ASC NULLS FIRST, store_name#202 ASC NULLS FIRST, store_zip#203 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#117) +Right keys [3]: [item_sk#201, store_name#202, store_zip#203] +Join condition: (cnt#205 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#206, s2#207, s3#208, syear#204, cnt#205] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#201, store_name#202, store_zip#203, syear#204, cnt#205, s1#206, s2#207, s3#208] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#206, s2#207, s3#208, syear#204, cnt#205] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#205 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#206 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#210] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#206, s2#207, s3#208, syear#204, cnt#205] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#205 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#206 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1033,6 +1033,6 @@ ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#150, d_year#151] 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 2b44e5ed9d4d..f43d53ab4359 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 @@ -71,8 +71,8 @@ WholeStageCodegen (54) WholeStageCodegen (9) Sort [cs_item_sk] Project [cs_item_sk] - 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] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter Exchange [cs_item_sk] #6 WholeStageCodegen (8) @@ -248,8 +248,8 @@ WholeStageCodegen (54) WholeStageCodegen (35) Sort [cs_item_sk] Project [cs_item_sk] - 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] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2), true)),sale,refund,sum,sum,isEmpty] InputAdapter ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 InputAdapter