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 c1e629aca7df6..3b9c0705bc72c 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 @@ -307,6 +307,7 @@ class Analyzer(override val catalogManager: CatalogManager) ResolveBinaryArithmetic :: ResolveUnion :: typeCoercionRules ++ + Seq(ResolveWithCTE) ++ extendedResolutionRules : _*), Batch("Remove TempResolvedColumn", Once, RemoveTempResolvedColumn), Batch("Apply Char Padding", Once, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 627877f0dcab7..a67d85d8433fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -20,25 +20,60 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions.SubqueryExpression -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, With} +import org.apache.spark.sql.catalyst.plans.logical.{Command, CTERelationDef, CTERelationRef, InsertIntoDir, LogicalPlan, ParsedStatement, SubqueryAlias, UnresolvedWith, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf.{LEGACY_CTE_PRECEDENCE_POLICY, LegacyBehaviorPolicy} /** - * Analyze WITH nodes and substitute child plan with CTE definitions. + * Analyze WITH nodes and substitute child plan with CTE references or CTE definitions depending + * on the conditions below: + * 1. If in legacy mode, or if the query is a SQL command or DML statement, replace with CTE + * definitions, i.e., inline CTEs. + * 2. Otherwise, replace with CTE references `CTERelationRef`s. The decision to inline or not + * inline will be made later by the rule `InlineCTE` after query analysis. + * + * All the CTE definitions that are not inlined after this substitution will be grouped together + * under one `WithCTE` node for each of the main query and the subqueries. Any of the main query + * or the subqueries that do not contain CTEs or have had all CTEs inlined will obviously not have + * any `WithCTE` nodes. If any though, the `WithCTE` node will be in the same place as where the + * outermost `With` node once was. + * + * The CTE definitions in a `WithCTE` node are kept in the order of how they have been resolved. + * That means the CTE definitions are guaranteed to be in topological order base on their + * dependency for any valid CTE query (i.e., given CTE definitions A and B with B referencing A, + * A is guaranteed to appear before B). Otherwise, it must be an invalid user query, and an + * analysis exception will be thrown later by relation resolving rules. */ object CTESubstitution extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { - LegacyBehaviorPolicy.withName(conf.getConf(LEGACY_CTE_PRECEDENCE_POLICY)) match { - case LegacyBehaviorPolicy.EXCEPTION => - assertNoNameConflictsInCTE(plan) - traverseAndSubstituteCTE(plan) - case LegacyBehaviorPolicy.LEGACY => - legacyTraverseAndSubstituteCTE(plan) - case LegacyBehaviorPolicy.CORRECTED => - traverseAndSubstituteCTE(plan) + val isCommand = plan.find { + case _: Command | _: ParsedStatement | _: InsertIntoDir => true + case _ => false + }.isDefined + val cteDefs = mutable.ArrayBuffer.empty[CTERelationDef] + val (substituted, lastSubstituted) = + LegacyBehaviorPolicy.withName(conf.getConf(LEGACY_CTE_PRECEDENCE_POLICY)) match { + case LegacyBehaviorPolicy.EXCEPTION => + assertNoNameConflictsInCTE(plan) + traverseAndSubstituteCTE(plan, isCommand, cteDefs) + case LegacyBehaviorPolicy.LEGACY => + (legacyTraverseAndSubstituteCTE(plan, cteDefs), None) + case LegacyBehaviorPolicy.CORRECTED => + traverseAndSubstituteCTE(plan, isCommand, cteDefs) + } + if (cteDefs.isEmpty) { + substituted + } else if (substituted eq lastSubstituted.get) { + WithCTE(substituted, cteDefs.toSeq) + } else { + var done = false + substituted.resolveOperatorsWithPruning(_ => !done) { + case p if p eq lastSubstituted.get => + done = true + WithCTE(p, cteDefs.toSeq) + } } } @@ -59,7 +94,7 @@ object CTESubstitution extends Rule[LogicalPlan] { startOfQuery: Boolean = true): Unit = { val resolver = conf.resolver plan match { - case With(child, relations) => + case UnresolvedWith(child, relations) => val newNames = mutable.ArrayBuffer.empty[String] newNames ++= outerCTERelationNames relations.foreach { @@ -81,17 +116,21 @@ object CTESubstitution extends Rule[LogicalPlan] { } } - private def legacyTraverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { + private def legacyTraverseAndSubstituteCTE( + plan: LogicalPlan, + cteDefs: mutable.ArrayBuffer[CTERelationDef]): LogicalPlan = { plan.resolveOperatorsUp { - case With(child, relations) => - val resolvedCTERelations = resolveCTERelations(relations, isLegacy = true) - substituteCTE(child, resolvedCTERelations) + case UnresolvedWith(child, relations) => + val resolvedCTERelations = + resolveCTERelations(relations, isLegacy = true, isCommand = false, cteDefs) + substituteCTE(child, alwaysInline = true, resolvedCTERelations) } } /** - * Traverse the plan and expression nodes as a tree and replace matching references to CTE - * definitions. + * Traverse the plan and expression nodes as a tree and replace matching references with CTE + * references if `isCommand` is false, otherwise with the query plans of the corresponding + * CTE definitions. * - If the rule encounters a WITH node then it substitutes the child of the node with CTE * definitions of the node right-to-left order as a definition can reference to a previous * one. @@ -130,23 +169,36 @@ object CTESubstitution extends Rule[LogicalPlan] { * @param plan the plan to be traversed * @return the plan where CTE substitution is applied */ - private def traverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { - plan.resolveOperatorsUpWithPruning(_.containsAnyPattern(UNRESOLVED_RELATION, PLAN_EXPRESSION)) { - case With(child: LogicalPlan, relations) => - val resolvedCTERelations = resolveCTERelations(relations, isLegacy = false) - substituteCTE(child, resolvedCTERelations) + private def traverseAndSubstituteCTE( + plan: LogicalPlan, + isCommand: Boolean, + cteDefs: mutable.ArrayBuffer[CTERelationDef]): (LogicalPlan, Option[LogicalPlan]) = { + var lastSubstituted: Option[LogicalPlan] = None + val newPlan = plan.resolveOperatorsUpWithPruning( + _.containsAnyPattern(UNRESOLVED_RELATION, PLAN_EXPRESSION)) { + case UnresolvedWith(child: LogicalPlan, relations) => + val resolvedCTERelations = + resolveCTERelations(relations, isLegacy = false, isCommand, cteDefs) + if (!isCommand) { + cteDefs ++= resolvedCTERelations.map(_._2) + } + lastSubstituted = Some(substituteCTE(child, isCommand, resolvedCTERelations)) + lastSubstituted.get case other => other.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan)) + case e: SubqueryExpression => e.withNewPlan(apply(e.plan)) } } + (newPlan, lastSubstituted) } private def resolveCTERelations( relations: Seq[(String, SubqueryAlias)], - isLegacy: Boolean): Seq[(String, LogicalPlan)] = { - val resolvedCTERelations = new mutable.ArrayBuffer[(String, LogicalPlan)](relations.size) + isLegacy: Boolean, + isCommand: Boolean, + cteDefs: mutable.ArrayBuffer[CTERelationDef]): Seq[(String, CTERelationDef)] = { + val resolvedCTERelations = new mutable.ArrayBuffer[(String, CTERelationDef)](relations.size) for ((name, relation) <- relations) { val innerCTEResolved = if (isLegacy) { // In legacy mode, outer CTE relations take precedence. Here we don't resolve the inner @@ -156,25 +208,37 @@ object CTESubstitution extends Rule[LogicalPlan] { } else { // A CTE definition might contain an inner CTE that has a higher priority, so traverse and // substitute CTE defined in `relation` first. - traverseAndSubstituteCTE(relation) + traverseAndSubstituteCTE(relation, isCommand, cteDefs)._1 } // CTE definition can reference a previous one - resolvedCTERelations += (name -> substituteCTE(innerCTEResolved, resolvedCTERelations.toSeq)) + val substituted = + substituteCTE(innerCTEResolved, isLegacy || isCommand, resolvedCTERelations.toSeq) + val cteRelation = CTERelationDef(substituted) + resolvedCTERelations += (name -> cteRelation) } resolvedCTERelations.toSeq } private def substituteCTE( plan: LogicalPlan, - cteRelations: Seq[(String, LogicalPlan)]): LogicalPlan = + alwaysInline: Boolean, + cteRelations: Seq[(String, CTERelationDef)]): LogicalPlan = plan.resolveOperatorsUpWithPruning(_.containsAnyPattern(UNRESOLVED_RELATION, PLAN_EXPRESSION)) { case u @ UnresolvedRelation(Seq(table), _, _) => - cteRelations.find(r => plan.conf.resolver(r._1, table)).map(_._2).getOrElse(u) + cteRelations.find(r => plan.conf.resolver(r._1, table)).map { case (_, d) => + if (alwaysInline) { + d.child + } else { + // Add a `SubqueryAlias` for hint-resolving rules to match relation names. + SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output)) + } + }.getOrElse(u) case other => // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. other.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case e: SubqueryExpression => e.withNewPlan(substituteCTE(e.plan, cteRelations)) + case e: SubqueryExpression => + e.withNewPlan(apply(substituteCTE(e.plan, alwaysInline, cteRelations))) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index 569c361c8d616..27f2a5f416d56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -120,7 +120,7 @@ object ResolveHints { case r: SubqueryAlias if matchedIdentifierInHint(extractIdentifier(r)) => ResolvedHint(plan, createHintInfo(hintName)) - case _: ResolvedHint | _: View | _: With | _: SubqueryAlias => + case _: ResolvedHint | _: View | _: UnresolvedWith | _: SubqueryAlias => // Don't traverse down these nodes. // For an existing strategy hint, there is no chance for a match from this point down. // The rest (view, with, subquery) indicates different scopes that we shouldn't traverse diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala new file mode 100644 index 0000000000000..78b776f12f074 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LogicalPlan, WithCTE} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} + +/** + * Updates CTE references with the resolve output attributes of corresponding CTE definitions. + */ +object ResolveWithCTE extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + if (plan.containsAllPatterns(CTE)) { + val cteDefMap = mutable.HashMap.empty[Long, CTERelationDef] + resolveWithCTE(plan, cteDefMap) + } else { + plan + } + } + + private def resolveWithCTE( + plan: LogicalPlan, + cteDefMap: mutable.HashMap[Long, CTERelationDef]): LogicalPlan = { + plan.resolveOperatorsDownWithPruning(_.containsAllPatterns(CTE)) { + case w @ WithCTE(_, cteDefs) => + cteDefs.foreach { cteDef => + if (cteDef.resolved) { + cteDefMap.put(cteDef.id, cteDef) + } + } + w + + case ref: CTERelationRef if !ref.resolved => + cteDefMap.get(ref.cteId).map { cteDef => + CTERelationRef(cteDef.id, cteDef.resolved, cteDef.output) + }.getOrElse { + ref + } + + case other => + other.transformExpressionsWithPruning(_.containsAllPatterns(PLAN_EXPRESSION, CTE)) { + case e: SubqueryExpression => e.withNewPlan(resolveWithCTE(e.plan, cteDefMap)) + } + } + } +} 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 0c7452a37d54a..8918bb3ea86f3 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 @@ -76,6 +76,7 @@ abstract class SubqueryExpression( AttributeSet.fromAttributeSets(outerAttrs.map(_.references)) override def children: Seq[Expression] = outerAttrs ++ joinCond override def withNewPlan(plan: LogicalPlan): SubqueryExpression + def isCorrelated: Boolean = outerAttrs.nonEmpty } object SubqueryExpression { @@ -86,7 +87,7 @@ object SubqueryExpression { def hasInOrCorrelatedExistsSubquery(e: Expression): Boolean = { e.find { case _: ListQuery => true - case _: Exists if e.children.nonEmpty => true + case ex: Exists => ex.isCorrelated case _ => false }.isDefined } @@ -98,7 +99,7 @@ object SubqueryExpression { */ def hasCorrelatedSubquery(e: Expression): Boolean = { e.find { - case s: SubqueryExpression => s.children.nonEmpty + case s: SubqueryExpression => s.isCorrelated case _ => false }.isDefined } @@ -279,7 +280,7 @@ case class ScalarSubquery( object ScalarSubquery { def hasCorrelatedScalarSubquery(e: Expression): Boolean = { e.find { - case s: ScalarSubquery => s.children.nonEmpty + case s: ScalarSubquery => s.isCorrelated case _ => false }.isDefined } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala new file mode 100644 index 0000000000000..6bcbc9f821de6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations +import org.apache.spark.sql.catalyst.expressions.{Alias, OuterReference, SubqueryExpression} +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, Join, JoinHint, LogicalPlan, Project, Subquery, WithCTE} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{CTE, PLAN_EXPRESSION} + +/** + * Inlines CTE definitions into corresponding references if either of the conditions satisfies: + * 1. The CTE definition does not contain any non-deterministic expressions. If this CTE + * definition references another CTE definition that has non-deterministic expressions, it + * is still OK to inline the current CTE definition. + * 2. The CTE definition is only referenced once throughout the main query and all the subqueries. + * + * In addition, due to the complexity of correlated subqueries, all CTE references in correlated + * subqueries are inlined regardless of the conditions above. + */ +object InlineCTE extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + if (!plan.isInstanceOf[Subquery] && plan.containsPattern(CTE)) { + val cteMap = mutable.HashMap.empty[Long, (CTERelationDef, Int)] + buildCTEMap(plan, cteMap) + inlineCTE(plan, cteMap, forceInline = false) + } else { + plan + } + } + + private def shouldInline(cteDef: CTERelationDef, refCount: Int): Boolean = { + // We do not need to check enclosed `CTERelationRef`s for `deterministic` or `OuterReference`, + // because: + // 1) It is fine to inline a CTE if it references another CTE that is non-deterministic; + // 2) Any `CTERelationRef` that contains `OuterReference` would have been inlined first. + refCount == 1 || + cteDef.child.find(_.expressions.exists(!_.deterministic)).isEmpty || + cteDef.child.find(_.expressions.exists(_.isInstanceOf[OuterReference])).isDefined + } + + private def buildCTEMap( + plan: LogicalPlan, + cteMap: mutable.HashMap[Long, (CTERelationDef, Int)]): Unit = { + plan match { + case WithCTE(_, cteDefs) => + cteDefs.foreach { cteDef => + cteMap.put(cteDef.id, (cteDef, 0)) + } + + case ref: CTERelationRef => + val (cteDef, refCount) = cteMap(ref.cteId) + cteMap.update(ref.cteId, (cteDef, refCount + 1)) + + case _ => + } + + if (plan.containsPattern(CTE)) { + plan.children.foreach { child => + buildCTEMap(child, cteMap) + } + + plan.expressions.foreach { expr => + if (expr.containsAllPatterns(PLAN_EXPRESSION, CTE)) { + expr.foreach { + case e: SubqueryExpression => + buildCTEMap(e.plan, cteMap) + case _ => + } + } + } + } + } + + private def inlineCTE( + plan: LogicalPlan, + cteMap: mutable.HashMap[Long, (CTERelationDef, Int)], + forceInline: Boolean): LogicalPlan = { + val (stripped, notInlined) = plan match { + case WithCTE(child, cteDefs) => + val notInlined = mutable.ArrayBuffer.empty[CTERelationDef] + cteDefs.foreach { cteDef => + val (cte, refCount) = cteMap(cteDef.id) + if (refCount > 0) { + val inlined = cte.copy(child = inlineCTE(cte.child, cteMap, forceInline)) + cteMap.update(cteDef.id, (inlined, refCount)) + if (!forceInline && !shouldInline(inlined, refCount)) { + notInlined.append(inlined) + } + } + } + (inlineCTE(child, cteMap, forceInline), notInlined.toSeq) + + case ref: CTERelationRef => + val (cteDef, refCount) = cteMap(ref.cteId) + val newRef = if (forceInline || shouldInline(cteDef, refCount)) { + if (ref.outputSet == cteDef.outputSet) { + cteDef.child + } else { + val ctePlan = DeduplicateRelations( + Join(cteDef.child, cteDef.child, Inner, None, JoinHint(None, None))).children(1) + val projectList = ref.output.zip(ctePlan.output).map { case (tgtAttr, srcAttr) => + Alias(srcAttr, tgtAttr.name)(exprId = tgtAttr.exprId) + } + Project(projectList, ctePlan) + } + } else { + ref + } + (newRef, Seq.empty) + + case _ if plan.containsPattern(CTE) => + val newPlan = plan + .withNewChildren(plan.children.map(child => inlineCTE(child, cteMap, forceInline))) + .transformExpressionsWithPruning(_.containsAllPatterns(PLAN_EXPRESSION, CTE)) { + case e: SubqueryExpression => + e.withNewPlan(inlineCTE(e.plan, cteMap, forceInline = e.isCorrelated)) + } + (newPlan, Seq.empty) + + case _ => (plan, Seq.empty) + } + + if (notInlined.isEmpty) { + stripped + } else { + WithCTE(stripped, notInlined) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 96834251e4e7b..80955deed18c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -153,6 +153,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateResolvedHint, EliminateSubqueryAliases, EliminateView, + InlineCTE, ReplaceExpressions, RewriteNonCorrelatedExists, PullOutGroupingExpressions, @@ -209,6 +210,7 @@ abstract class Optimizer(catalogManager: CatalogManager) // plan may contain nodes that do not report stats. Anything that uses stats must run after // this batch. Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*) :+ + Batch("Update CTE Relation Stats", Once, UpdateCTERelationStats) :+ // Since join costs in AQP can change between multiple runs, there is no reason that we have an // idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once. Batch("Join Reorder", FixedPoint(1), @@ -305,6 +307,42 @@ abstract class Optimizer(catalogManager: CatalogManager) } } + /** + * Update CTE reference stats. + */ + object UpdateCTERelationStats extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + if (!plan.isInstanceOf[Subquery] && plan.containsPattern(CTE)) { + val statsMap = mutable.HashMap.empty[Long, Statistics] + updateCTEStats(plan, statsMap) + } else { + plan + } + } + + private def updateCTEStats( + plan: LogicalPlan, + statsMap: mutable.HashMap[Long, Statistics]): LogicalPlan = plan match { + case WithCTE(child, cteDefs) => + val newDefs = cteDefs.map { cteDef => + val newDef = updateCTEStats(cteDef, statsMap) + statsMap.put(cteDef.id, newDef.stats) + newDef.asInstanceOf[CTERelationDef] + } + WithCTE(updateCTEStats(child, statsMap), newDefs) + case c: CTERelationRef => + statsMap.get(c.cteId).map(s => c.withNewStats(Some(s))).getOrElse(c) + case _ if plan.containsPattern(CTE) => + plan + .withNewChildren(plan.children.map(child => updateCTEStats(child, statsMap))) + .transformExpressionsWithPruning(_.containsAllPatterns(PLAN_EXPRESSION, CTE)) { + case e: SubqueryExpression => + e.withNewPlan(updateCTEStats(e.plan, statsMap)) + } + case _ => plan + } + } + /** * Override to provide additional rules for the operator optimization batch. */ @@ -815,6 +853,14 @@ object ColumnPruning extends Rule[LogicalPlan] { p.copy(child = w.copy( windowExpressions = w.windowExpressions.filter(p.references.contains))) + // Prune WithCTE + case p @ Project(_, w: WithCTE) => + if (!w.outputSet.subsetOf(p.references)) { + p.copy(child = w.withNewPlan(prunedChild(w.plan, p.references))) + } else { + p + } + // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 9ff99d50d4de3..39b202d6ef138 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -137,7 +137,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg throw QueryParsingErrors.duplicateCteDefinitionNamesError( duplicates.mkString("'", "', '", "'"), ctx) } - With(plan, ctes.toSeq) + UnresolvedWith(plan, ctes.toSeq) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala index 20398dd40b209..ba927746bbf6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala @@ -43,6 +43,7 @@ trait LogicalPlanVisitor[T] { case p: Window => visitWindow(p) case p: Tail => visitTail(p) case p: Sort => visitSort(p) + case p: WithCTE => visitWithCTE(p) case p: LogicalPlan => default(p) } @@ -87,4 +88,6 @@ trait LogicalPlanVisitor[T] { def visitTail(p: Tail): T def visitSort(sort: Sort): T + + def visitWithCTE(p: WithCTE): T } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index d9efdc122c9ef..e456c5d05323f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -615,7 +615,9 @@ object View { * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined * Each CTE can see the base tables and the previously defined CTEs only. */ -case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { +case class UnresolvedWith( + child: LogicalPlan, + cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { override def output: Seq[Attribute] = child.output override def simpleString(maxFields: Int): String = { @@ -625,7 +627,78 @@ case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) override def innerChildren: Seq[LogicalPlan] = cteRelations.map(_._2) - override protected def withNewChildInternal(newChild: LogicalPlan): With = copy(child = newChild) + override protected def withNewChildInternal(newChild: LogicalPlan): UnresolvedWith = + copy(child = newChild) +} + +/** + * A wrapper for CTE definition plan with a unique ID. + * @param child The CTE definition query plan. + * @param id The unique ID for this CTE definition. + */ +case class CTERelationDef(child: LogicalPlan, id: Long = CTERelationDef.newId) extends UnaryNode { + + final override val nodePatterns: Seq[TreePattern] = Seq(CTE) + + override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = + copy(child = newChild) + + override def output: Seq[Attribute] = if (resolved) child.output else Nil +} + +object CTERelationDef { + private val curId = new java.util.concurrent.atomic.AtomicLong() + def newId: Long = curId.getAndIncrement() +} + +/** + * Represents the relation of a CTE reference. + * @param cteId The ID of the corresponding CTE definition. + * @param _resolved Whether this reference is resolved. + * @param output The output attributes of this CTE reference, which can be different from + * the output of its corresponding CTE definition after attribute de-duplication. + * @param statsOpt The optional statistics inferred from the corresponding CTE definition. + */ +case class CTERelationRef( + cteId: Long, + _resolved: Boolean, + override val output: Seq[Attribute], + statsOpt: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { + + final override val nodePatterns: Seq[TreePattern] = Seq(CTE) + + override lazy val resolved: Boolean = _resolved + + override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) + + def withNewStats(statsOpt: Option[Statistics]): CTERelationRef = copy(statsOpt = statsOpt) + + override def computeStats(): Statistics = statsOpt.getOrElse(Statistics(conf.defaultSizeInBytes)) +} + +/** + * The resolved version of [[UnresolvedWith]] with CTE referrences linked to CTE definitions + * through unique IDs instead of relation aliases. + * + * @param plan The query plan. + * @param cteDefs The CTE definitions. + */ +case class WithCTE(plan: LogicalPlan, cteDefs: Seq[CTERelationDef]) extends LogicalPlan { + + final override val nodePatterns: Seq[TreePattern] = Seq(CTE) + + override def output: Seq[Attribute] = plan.output + + override def children: Seq[LogicalPlan] = cteDefs :+ plan + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + copy(plan = newChildren.last, cteDefs = newChildren.init.asInstanceOf[Seq[CTERelationDef]]) + } + + def withNewPlan(newPlan: LogicalPlan): WithCTE = { + withNewChildren(children.init :+ newPlan).asInstanceOf[WithCTE] + } } case class WithWindowDefinition( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala index 67e4ad0f206ae..3f702724cca53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala @@ -105,4 +105,6 @@ object BasicStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { override def visitTail(p: Tail): Statistics = { fallback(p) } + + override def visitWithCTE(p: WithCTE): Statistics = fallback(p) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala index 5c5ccdea55c79..73c1b9445f693 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala @@ -161,4 +161,6 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { sizeInBytes = EstimationUtils.getOutputSize(p.output, rowCount, childStats.attributeStats), rowCount = Some(rowCount)) } + + override def visitWithCTE(p: WithCTE): Statistics = p.plan.stats } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 8425bfff070e9..fc9031b425ea1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -92,6 +92,7 @@ object TreePattern extends Enumeration { // Logical plan patterns (alphabetically ordered) val AGGREGATE: Value = Value val COMMAND: Value = Value + val CTE: Value = Value val DISTINCT_LIKE: Value = Value val EVENT_TIME_WATERMARK: Value = Value val EXCEPT: Value = Value diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 6cda05360aea3..960c260962772 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -714,7 +714,7 @@ class AnalysisErrorSuite extends AnalysisTest { test("SPARK-30811: CTE should not cause stack overflow when " + "it refers to non-existent table with same name") { - val plan = With( + val plan = UnresolvedWith( UnresolvedRelation(TableIdentifier("t")), Seq("t" -> SubqueryAlias("t", Project( 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 6ddc6b7eb716b..d3dd3acff9f0d 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 @@ -952,7 +952,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-32237: Hint in CTE") { - val plan = With( + val plan = UnresolvedWith( Project( Seq(UnresolvedAttribute("cte.a")), UnresolvedRelation(TableIdentifier("cte")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 8332668a754e1..246538438c07a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{QueryPlanningTracker, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog, TemporaryViewRelation} import org.apache.spark.sql.catalyst.catalog.CatalogTable.VIEW_STORING_ANALYZED_PLAN +import org.apache.spark.sql.catalyst.optimizer.InlineCTE import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ @@ -99,13 +100,19 @@ trait AnalysisTest extends PlanTest { protected def checkAnalysisWithoutViewWrapper( inputPlan: LogicalPlan, expectedPlan: LogicalPlan, - caseSensitive: Boolean = true): Unit = { + caseSensitive: Boolean = true, + inlineCTE: Boolean = false): Unit = { withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - val actualPlan = getAnalyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) - val transformed = EliminateSubqueryAliases(actualPlan) transformUp { + val analyzed = getAnalyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) + val transformed = EliminateSubqueryAliases(analyzed) transformUp { case v: View if v.isTempViewStoringAnalyzedPlan => v.child } - comparePlans(transformed, expectedPlan) + val actualPlan = if (inlineCTE) { + InlineCTE(transformed) + } else { + transformed + } + comparePlans(actualPlan, expectedPlan) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 9fa0f78df8024..dad14d3333d10 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -113,7 +113,8 @@ class ResolveHintsSuite extends AnalysisTest { ), ResolvedHint(testRelation.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) .select('a).analyze, - caseSensitive = false) + caseSensitive = false, + inlineCTE = true) } test("should not traverse down CTE") { @@ -125,7 +126,8 @@ class ResolveHintsSuite extends AnalysisTest { """.stripMargin ), testRelation.where('a > 1).select('a).select('a).analyze, - caseSensitive = false) + caseSensitive = false, + inlineCTE = true) } test("coalesce and repartition hint") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 45f865f565024..a1d9f894a157e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1513,7 +1513,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin, MergeIntoTable( SubqueryAlias("target", UnresolvedRelation(Seq("testcat1", "ns1", "ns2", "tbl"))), - SubqueryAlias("source", With(Project(Seq(UnresolvedStar(None)), + SubqueryAlias("source", UnresolvedWith(Project(Seq(UnresolvedStar(None)), UnresolvedRelation(Seq("s"))), Seq("s" -> SubqueryAlias("s", Project(Seq(UnresolvedStar(None)), UnresolvedRelation(Seq("testcat2", "ns1", "ns2", "tbl"))))))), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index e1b484083de47..ebafb9db1be32 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -42,7 +42,9 @@ class PlanParserSuite extends AnalysisTest { private def intercept(sqlCommand: String, messages: String*): Unit = interceptParseException(parsePlan)(sqlCommand, messages: _*) - private def cte(plan: LogicalPlan, namedPlans: (String, (LogicalPlan, Seq[String]))*): With = { + private def cte( + plan: LogicalPlan, + namedPlans: (String, (LogicalPlan, Seq[String]))*): UnresolvedWith = { val ctes = namedPlans.map { case (name, (cte, columnAliases)) => val subquery = if (columnAliases.isEmpty) { @@ -52,7 +54,7 @@ class PlanParserSuite extends AnalysisTest { } name -> SubqueryAlias(name, subquery) } - With(plan, ctes) + UnresolvedWith(plan, ctes) } test("single comment case one") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 361a9103eb03c..bb1b0ca3b645a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -21,6 +21,8 @@ import java.io.{BufferedWriter, OutputStreamWriter} import java.util.UUID import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable + import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging @@ -30,7 +32,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, CTERelationDef, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString @@ -62,6 +64,17 @@ class QueryExecution( // TODO: Move the planner an optimizer into here from SessionState. protected def planner = sparkSession.sessionState.planner + // The CTE map for the planner shared by the main query and all subqueries. + private val cteMap = mutable.HashMap.empty[Long, CTERelationDef] + + def withCteMap[T](f: => T): T = { + val old = QueryExecution.currentCteMap.get() + QueryExecution.currentCteMap.set(cteMap) + try f finally { + QueryExecution.currentCteMap.set(old) + } + } + def assertAnalyzed(): Unit = analyzed def assertSupported(): Unit = { @@ -134,7 +147,7 @@ class QueryExecution( private def assertOptimized(): Unit = optimizedPlan - lazy val sparkPlan: SparkPlan = { + lazy val sparkPlan: SparkPlan = withCteMap { // We need to materialize the optimizedPlan here because sparkPlan is also tracked under // the planning phase assertOptimized() @@ -147,7 +160,7 @@ class QueryExecution( // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = { + lazy val executedPlan: SparkPlan = withCteMap { // We need to materialize the optimizedPlan here, before tracking the planning phase, to ensure // that the optimization time is not counted as part of the planning phase. assertOptimized() @@ -470,4 +483,8 @@ object QueryExecution { val sparkPlan = createSparkPlan(spark, spark.sessionState.planner, plan.clone()) prepareExecutedPlan(spark, sparkPlan) } + + private val currentCteMap = new ThreadLocal[mutable.HashMap[Long, CTERelationDef]]() + + def cteMap: mutable.HashMap[Long, CTERelationDef] = currentCteMap.get() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index 6994aaf47dfba..32ac58f8353ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -44,6 +44,7 @@ class SparkPlanner(val session: SparkSession, val experimentalMethods: Experimen JoinSelection :: InMemoryScans :: SparkScripts :: + WithCTEStrategy :: BasicOperators :: Nil) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 6ebfba2c02957..931e9865b4ec7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.physical.RoundRobinPartitioning import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.aggregate.AggUtils @@ -662,6 +663,36 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + /** + * Strategy to plan CTE relations left not inlined. + */ + object WithCTEStrategy extends Strategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case WithCTE(plan, cteDefs) => + val cteMap = QueryExecution.cteMap + cteDefs.foreach { cteDef => + cteMap.put(cteDef.id, cteDef) + } + planLater(plan) :: Nil + + case r: CTERelationRef => + val ctePlan = QueryExecution.cteMap(r.cteId).child + val projectList = r.output.zip(ctePlan.output).map { case (tgtAttr, srcAttr) => + Alias(srcAttr, tgtAttr.name)(exprId = tgtAttr.exprId) + } + val newPlan = Project(projectList, ctePlan) + // Plan CTE ref as a repartition shuffle so that all refs of the same CTE def will share + // an Exchange reuse at runtime. + // TODO create a new identity partitioning instead of using RoundRobinPartitioning. + exchange.ShuffleExchangeExec( + RoundRobinPartitioning(conf.numShufflePartitions), + planLater(newPlan), + REPARTITION_BY_COL) :: Nil + + case _ => Nil + } + } + object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case d: DataWritingCommand => DataWritingCommandExec(d, planLater(d.query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index cd47fd016b6fe..bf810f3f65877 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -138,7 +138,9 @@ case class AdaptiveSparkPlanExec( collapseCodegenStagesRule ) - private def optimizeQueryStage(plan: SparkPlan, isFinalStage: Boolean): SparkPlan = { + private def optimizeQueryStage( + plan: SparkPlan, + isFinalStage: Boolean): SparkPlan = context.qe.withCteMap { val optimized = queryStageOptimizerRules.foldLeft(plan) { case (latestPlan, rule) => val applied = rule.apply(latestPlan) val result = rule match { @@ -627,11 +629,6 @@ case class AdaptiveSparkPlanExec( val newLogicalPlan = logicalPlan.transformDown { case p if p.eq(logicalNode) => newLogicalNode } - assert(newLogicalPlan != logicalPlan, - s"logicalNode: $logicalNode; " + - s"logicalPlan: $logicalPlan " + - s"physicalPlan: $currentPhysicalPlan" + - s"stage: $stage") logicalPlan = newLogicalPlan case _ => // Ignore those earlier stages that have been wrapped in later stages. @@ -642,7 +639,8 @@ case class AdaptiveSparkPlanExec( /** * Re-optimize and run physical planning on the current logical plan based on the latest stats. */ - private def reOptimize(logicalPlan: LogicalPlan): (SparkPlan, LogicalPlan) = { + private def reOptimize( + logicalPlan: LogicalPlan): (SparkPlan, LogicalPlan) = context.qe.withCteMap { logicalPlan.invalidateStatsCache() val optimized = optimizer.execute(logicalPlan) val sparkPlan = context.session.sessionState.planner.plan(ReturnAnswer(optimized)).next() diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index 21bad134706bb..951d61cc2788b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out @@ -21,7 +21,7 @@ SELECT count(*) FROM ( -- !query schema struct -- !query output -10 +5 -- !query diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt index 0dc66338ffc3b..65572cfd770d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt @@ -205,200 +205,200 @@ Aggregate Attributes [4]: [avg(agg1#22)#43, avg(UnscaledValue(agg2#23))#44, avg( Results [7]: [i_item_id#20, s_state#17, 0 AS g_state#47, avg(agg1#22)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#23))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#24))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#25))#46 / 100.0) as decimal(11,6)) AS agg4#51] (29) Scan parquet default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), (ss_sold_date_sk#59 >= 2451545), (ss_sold_date_sk#59 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), (ss_sold_date_sk#8 >= 2451545), (ss_sold_date_sk#8 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (31) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (32) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#60] +Output [1]: [d_date_sk#10] (33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#60] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (34) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#60] +Output [7]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#10] (35) Scan parquet default.store -Output [2]: [s_store_sk#61, s_state#62] +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [AL,SD,TN]), IsNotNull(s_store_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#61, s_state#62] +Input [2]: [s_store_sk#16, s_state#17] (37) Filter [codegen id : 8] -Input [2]: [s_store_sk#61, s_state#62] -Condition : (s_state#62 IN (TN,AL,SD) AND isnotnull(s_store_sk#61)) +Input [2]: [s_store_sk#16, s_state#17] +Condition : (s_state#17 IN (TN,AL,SD) AND isnotnull(s_store_sk#16)) (38) Project [codegen id : 8] -Output [1]: [s_store_sk#61] -Input [2]: [s_store_sk#61, s_state#62] +Output [1]: [s_store_sk#16] +Input [2]: [s_store_sk#16, s_state#17] (39) BroadcastExchange -Input [1]: [s_store_sk#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [s_store_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#61] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_cdemo_sk#53, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#61] +Output [6]: [ss_item_sk#1, ss_cdemo_sk#2, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] (42) ReusedExchange [Reuses operator id: 11] -Output [1]: [cd_demo_sk#64] +Output [1]: [cd_demo_sk#11] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (44) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_cdemo_sk#53, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, cd_demo_sk#64] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_cdemo_sk#2, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, cd_demo_sk#11] (45) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#65, i_item_id#66] +Output [2]: [i_item_sk#19, i_item_id#20] (46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#65] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (47) Project [codegen id : 11] -Output [5]: [i_item_id#66, ss_quantity#55 AS agg1#22, ss_list_price#56 AS agg2#23, ss_coupon_amt#58 AS agg3#24, ss_sales_price#57 AS agg4#25] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#66] +Output [5]: [i_item_id#20, ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19, i_item_id#20] (48) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#66, agg1#22, agg2#23, agg3#24, agg4#25] -Keys [1]: [i_item_id#66] +Input [5]: [i_item_id#20, agg1#22, agg2#23, agg3#24, agg4#25] +Keys [1]: [i_item_id#20] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74] -Results [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] +Aggregate Attributes [8]: [sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Results [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] (49) Exchange -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Arguments: hashpartitioning(i_item_id#66, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#69] (50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Keys [1]: [i_item_id#66] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Keys [1]: [i_item_id#20] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#84, avg(UnscaledValue(agg2#23))#85, avg(UnscaledValue(agg3#24))#86, avg(UnscaledValue(agg4#25))#87] -Results [7]: [i_item_id#66, null AS s_state#88, 1 AS g_state#89, avg(agg1#22)#84 AS agg1#90, cast((avg(UnscaledValue(agg2#23))#85 / 100.0) as decimal(11,6)) AS agg2#91, cast((avg(UnscaledValue(agg3#24))#86 / 100.0) as decimal(11,6)) AS agg3#92, cast((avg(UnscaledValue(agg4#25))#87 / 100.0) as decimal(11,6)) AS agg4#93] +Aggregate Attributes [4]: [avg(agg1#22)#70, avg(UnscaledValue(agg2#23))#71, avg(UnscaledValue(agg3#24))#72, avg(UnscaledValue(agg4#25))#73] +Results [7]: [i_item_id#20, null AS s_state#74, 1 AS g_state#75, avg(agg1#22)#70 AS agg1#76, cast((avg(UnscaledValue(agg2#23))#71 / 100.0) as decimal(11,6)) AS agg2#77, cast((avg(UnscaledValue(agg3#24))#72 / 100.0) as decimal(11,6)) AS agg3#78, cast((avg(UnscaledValue(agg4#25))#73 / 100.0) as decimal(11,6)) AS agg4#79] (51) Scan parquet default.store_sales -Output [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#101), (ss_sold_date_sk#101 >= 2451545), (ss_sold_date_sk#101 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), (ss_sold_date_sk#8 >= 2451545), (ss_sold_date_sk#8 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (53) Filter [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Condition : ((isnotnull(ss_cdemo_sk#95) AND isnotnull(ss_store_sk#96)) AND isnotnull(ss_item_sk#94)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (54) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#101] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [7]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [9]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, d_date_sk#102] +Output [7]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#10] (57) ReusedExchange [Reuses operator id: 39] -Output [1]: [s_store_sk#103] +Output [1]: [s_store_sk#16] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#96] -Right keys [1]: [s_store_sk#103] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (59) Project [codegen id : 17] -Output [6]: [ss_item_sk#94, ss_cdemo_sk#95, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, s_store_sk#103] +Output [6]: [ss_item_sk#1, ss_cdemo_sk#2, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] (60) ReusedExchange [Reuses operator id: 11] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#11] (61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (62) Project [codegen id : 17] -Output [5]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [7]: [ss_item_sk#94, ss_cdemo_sk#95, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, cd_demo_sk#104] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_cdemo_sk#2, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, cd_demo_sk#11] (63) Scan parquet default.item -Output [1]: [i_item_sk#105] +Output [1]: [i_item_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#105] +Input [1]: [i_item_sk#19] (65) Filter [codegen id : 16] -Input [1]: [i_item_sk#105] -Condition : isnotnull(i_item_sk#105) +Input [1]: [i_item_sk#19] +Condition : isnotnull(i_item_sk#19) (66) BroadcastExchange -Input [1]: [i_item_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#106] +Input [1]: [i_item_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#94] -Right keys [1]: [i_item_sk#105] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (68) Project [codegen id : 17] -Output [4]: [ss_quantity#97 AS agg1#22, ss_list_price#98 AS agg2#23, ss_coupon_amt#100 AS agg3#24, ss_sales_price#99 AS agg4#25] -Input [6]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, i_item_sk#105] +Output [4]: [ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19] (69) HashAggregate [codegen id : 17] Input [4]: [agg1#22, agg2#23, agg3#24, agg4#25] Keys: [] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Results [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Aggregate Attributes [8]: [sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88] +Results [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] (70) Exchange -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (71) HashAggregate [codegen id : 18] -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] Keys: [] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#124, avg(UnscaledValue(agg2#23))#125, avg(UnscaledValue(agg3#24))#126, avg(UnscaledValue(agg4#25))#127] -Results [7]: [null AS i_item_id#128, null AS s_state#129, 1 AS g_state#130, avg(agg1#22)#124 AS agg1#131, cast((avg(UnscaledValue(agg2#23))#125 / 100.0) as decimal(11,6)) AS agg2#132, cast((avg(UnscaledValue(agg3#24))#126 / 100.0) as decimal(11,6)) AS agg3#133, cast((avg(UnscaledValue(agg4#25))#127 / 100.0) as decimal(11,6)) AS agg4#134] +Aggregate Attributes [4]: [avg(agg1#22)#98, avg(UnscaledValue(agg2#23))#99, avg(UnscaledValue(agg3#24))#100, avg(UnscaledValue(agg4#25))#101] +Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(agg1#22)#98 AS agg1#105, cast((avg(UnscaledValue(agg2#23))#99 / 100.0) as decimal(11,6)) AS agg2#106, cast((avg(UnscaledValue(agg3#24))#100 / 100.0) as decimal(11,6)) AS agg3#107, cast((avg(UnscaledValue(agg4#25))#101 / 100.0) as decimal(11,6)) AS agg4#108] (72) Union @@ -417,29 +417,29 @@ BroadcastExchange (78) (74) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#135] +Output [2]: [d_date_sk#10, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_year#135] +Input [2]: [d_date_sk#10, d_year#109] (76) Filter [codegen id : 1] -Input [2]: [d_date_sk#10, d_year#135] -Condition : ((((isnotnull(d_year#135) AND (d_year#135 = 2000)) AND (d_date_sk#10 >= 2451545)) AND (d_date_sk#10 <= 2451910)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#10, d_year#109] +Condition : ((((isnotnull(d_year#109) AND (d_year#109 = 2000)) AND (d_date_sk#10 >= 2451545)) AND (d_date_sk#10 <= 2451910)) AND isnotnull(d_date_sk#10)) (77) Project [codegen id : 1] Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#135] +Input [2]: [d_date_sk#10, d_year#109] (78) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#136] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt index 5a7cbfa1a9a3c..0fe719015aaa3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt @@ -205,200 +205,200 @@ Aggregate Attributes [4]: [avg(agg1#22)#43, avg(UnscaledValue(agg2#23))#44, avg( Results [7]: [i_item_id#20, s_state#17, 0 AS g_state#47, avg(agg1#22)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#23))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#24))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#25))#46 / 100.0) as decimal(11,6)) AS agg4#51] (29) Scan parquet default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), (ss_sold_date_sk#59 >= 2451545), (ss_sold_date_sk#59 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), (ss_sold_date_sk#8 >= 2451545), (ss_sold_date_sk#8 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (31) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (32) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#60] +Output [1]: [cd_demo_sk#10] (33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#60] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (34) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (35) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#15] (36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (37) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#15] (38) Scan parquet default.store -Output [2]: [s_store_sk#62, s_state#63] +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [AL,SD,TN]), IsNotNull(s_store_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#62, s_state#63] +Input [2]: [s_store_sk#16, s_state#17] (40) Filter [codegen id : 9] -Input [2]: [s_store_sk#62, s_state#63] -Condition : (s_state#63 IN (TN,AL,SD) AND isnotnull(s_store_sk#62)) +Input [2]: [s_store_sk#16, s_state#17] +Condition : (s_state#17 IN (TN,AL,SD) AND isnotnull(s_store_sk#16)) (41) Project [codegen id : 9] -Output [1]: [s_store_sk#62] -Input [2]: [s_store_sk#62, s_state#63] +Output [1]: [s_store_sk#16] +Input [2]: [s_store_sk#16, s_state#17] (42) BroadcastExchange -Input [1]: [s_store_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#64] +Input [1]: [s_store_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#62] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (44) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] (45) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#65, i_item_id#66] +Output [2]: [i_item_sk#19, i_item_id#20] (46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#65] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (47) Project [codegen id : 11] -Output [5]: [i_item_id#66, ss_quantity#55 AS agg1#22, ss_list_price#56 AS agg2#23, ss_coupon_amt#58 AS agg3#24, ss_sales_price#57 AS agg4#25] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#66] +Output [5]: [i_item_id#20, ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19, i_item_id#20] (48) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#66, agg1#22, agg2#23, agg3#24, agg4#25] -Keys [1]: [i_item_id#66] +Input [5]: [i_item_id#20, agg1#22, agg2#23, agg3#24, agg4#25] +Keys [1]: [i_item_id#20] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74] -Results [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] +Aggregate Attributes [8]: [sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Results [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] (49) Exchange -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Arguments: hashpartitioning(i_item_id#66, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#69] (50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Keys [1]: [i_item_id#66] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Keys [1]: [i_item_id#20] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#84, avg(UnscaledValue(agg2#23))#85, avg(UnscaledValue(agg3#24))#86, avg(UnscaledValue(agg4#25))#87] -Results [7]: [i_item_id#66, null AS s_state#88, 1 AS g_state#89, avg(agg1#22)#84 AS agg1#90, cast((avg(UnscaledValue(agg2#23))#85 / 100.0) as decimal(11,6)) AS agg2#91, cast((avg(UnscaledValue(agg3#24))#86 / 100.0) as decimal(11,6)) AS agg3#92, cast((avg(UnscaledValue(agg4#25))#87 / 100.0) as decimal(11,6)) AS agg4#93] +Aggregate Attributes [4]: [avg(agg1#22)#70, avg(UnscaledValue(agg2#23))#71, avg(UnscaledValue(agg3#24))#72, avg(UnscaledValue(agg4#25))#73] +Results [7]: [i_item_id#20, null AS s_state#74, 1 AS g_state#75, avg(agg1#22)#70 AS agg1#76, cast((avg(UnscaledValue(agg2#23))#71 / 100.0) as decimal(11,6)) AS agg2#77, cast((avg(UnscaledValue(agg3#24))#72 / 100.0) as decimal(11,6)) AS agg3#78, cast((avg(UnscaledValue(agg4#25))#73 / 100.0) as decimal(11,6)) AS agg4#79] (51) Scan parquet default.store_sales -Output [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#101), (ss_sold_date_sk#101 >= 2451545), (ss_sold_date_sk#101 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), (ss_sold_date_sk#8 >= 2451545), (ss_sold_date_sk#8 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (53) Filter [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Condition : ((isnotnull(ss_cdemo_sk#95) AND isnotnull(ss_store_sk#96)) AND isnotnull(ss_item_sk#94)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (54) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#102] +Output [1]: [cd_demo_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#102] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Input [9]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, cd_demo_sk#102] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (57) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#15] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#101] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (59) Project [codegen id : 17] -Output [6]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [8]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, d_date_sk#103] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#15] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [s_store_sk#104] +Output [1]: [s_store_sk#16] (61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#96] -Right keys [1]: [s_store_sk#104] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (62) Project [codegen id : 17] -Output [5]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, s_store_sk#104] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] (63) Scan parquet default.item -Output [1]: [i_item_sk#105] +Output [1]: [i_item_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#105] +Input [1]: [i_item_sk#19] (65) Filter [codegen id : 16] -Input [1]: [i_item_sk#105] -Condition : isnotnull(i_item_sk#105) +Input [1]: [i_item_sk#19] +Condition : isnotnull(i_item_sk#19) (66) BroadcastExchange -Input [1]: [i_item_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#106] +Input [1]: [i_item_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#94] -Right keys [1]: [i_item_sk#105] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (68) Project [codegen id : 17] -Output [4]: [ss_quantity#97 AS agg1#22, ss_list_price#98 AS agg2#23, ss_coupon_amt#100 AS agg3#24, ss_sales_price#99 AS agg4#25] -Input [6]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, i_item_sk#105] +Output [4]: [ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19] (69) HashAggregate [codegen id : 17] Input [4]: [agg1#22, agg2#23, agg3#24, agg4#25] Keys: [] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Results [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Aggregate Attributes [8]: [sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88] +Results [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] (70) Exchange -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (71) HashAggregate [codegen id : 18] -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] Keys: [] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#124, avg(UnscaledValue(agg2#23))#125, avg(UnscaledValue(agg3#24))#126, avg(UnscaledValue(agg4#25))#127] -Results [7]: [null AS i_item_id#128, null AS s_state#129, 1 AS g_state#130, avg(agg1#22)#124 AS agg1#131, cast((avg(UnscaledValue(agg2#23))#125 / 100.0) as decimal(11,6)) AS agg2#132, cast((avg(UnscaledValue(agg3#24))#126 / 100.0) as decimal(11,6)) AS agg3#133, cast((avg(UnscaledValue(agg4#25))#127 / 100.0) as decimal(11,6)) AS agg4#134] +Aggregate Attributes [4]: [avg(agg1#22)#98, avg(UnscaledValue(agg2#23))#99, avg(UnscaledValue(agg3#24))#100, avg(UnscaledValue(agg4#25))#101] +Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(agg1#22)#98 AS agg1#105, cast((avg(UnscaledValue(agg2#23))#99 / 100.0) as decimal(11,6)) AS agg2#106, cast((avg(UnscaledValue(agg3#24))#100 / 100.0) as decimal(11,6)) AS agg3#107, cast((avg(UnscaledValue(agg4#25))#101 / 100.0) as decimal(11,6)) AS agg4#108] (72) Union @@ -417,29 +417,29 @@ BroadcastExchange (78) (74) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#135] +Output [2]: [d_date_sk#15, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), GreaterThanOrEqual(d_date_sk,2451545), LessThanOrEqual(d_date_sk,2451910), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_year#135] +Input [2]: [d_date_sk#15, d_year#109] (76) Filter [codegen id : 1] -Input [2]: [d_date_sk#15, d_year#135] -Condition : ((((isnotnull(d_year#135) AND (d_year#135 = 2000)) AND (d_date_sk#15 >= 2451545)) AND (d_date_sk#15 <= 2451910)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#15, d_year#109] +Condition : ((((isnotnull(d_year#109) AND (d_year#109 = 2000)) AND (d_date_sk#15 >= 2451545)) AND (d_date_sk#15 <= 2451910)) AND isnotnull(d_date_sk#15)) (77) Project [codegen id : 1] Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#135] +Input [2]: [d_date_sk#15, d_year#109] (78) BroadcastExchange Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#136] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt index d20f7cf4a2857..d0259db7280d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt @@ -171,122 +171,122 @@ Output [10]: [s_store_name#39 AS s_store_name1#44, d_week_seq#5 AS d_week_seq1#4 Input [11]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, s_store_name#39, d_week_seq#42] (26) Scan parquet default.store_sales -Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#56)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] -Condition : isnotnull(ss_store_sk#54) +Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] +Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#56] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#4] Join condition: None (31) Project [codegen id : 6] -Output [4]: [ss_store_sk#54, ss_sales_price#55, d_week_seq#58, d_day_name#59] -Input [6]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56, d_date_sk#57, d_week_seq#58, d_day_name#59] +Output [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] (32) HashAggregate [codegen id : 6] -Input [4]: [ss_store_sk#54, ss_sales_price#55, d_week_seq#58, d_day_name#59] -Keys [2]: [d_week_seq#58, ss_store_sk#54] -Functions [6]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))] -Aggregate Attributes [6]: [sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] -Results [8]: [d_week_seq#58, ss_store_sk#54, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [6]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))] +Aggregate Attributes [6]: [sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Results [8]: [d_week_seq#5, ss_store_sk#1, sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] (33) Exchange -Input [8]: [d_week_seq#58, ss_store_sk#54, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] -Arguments: hashpartitioning(d_week_seq#58, ss_store_sk#54, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [8]: [d_week_seq#5, ss_store_sk#1, sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [id=#66] (34) HashAggregate [codegen id : 9] -Input [8]: [d_week_seq#58, ss_store_sk#54, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] -Keys [2]: [d_week_seq#58, ss_store_sk#54] -Functions [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))] -Aggregate Attributes [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END))#73, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END))#74, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END))#75, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END))#76, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END))#77, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))#78] -Results [8]: [d_week_seq#58, ss_store_sk#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END))#73,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END))#74,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END))#75,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END))#76,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END))#77,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))#78,17,2) AS sat_sales#36] +Input [8]: [d_week_seq#5, ss_store_sk#1, sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))] +Aggregate Attributes [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29] +Results [8]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29,17,2) AS sat_sales#36] (35) Scan parquet default.store -Output [2]: [s_store_sk#79, s_store_id#80] +Output [2]: [s_store_sk#67, s_store_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (36) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#79, s_store_id#80] +Input [2]: [s_store_sk#67, s_store_id#68] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#79, s_store_id#80] -Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) +Input [2]: [s_store_sk#67, s_store_id#68] +Condition : (isnotnull(s_store_sk#67) AND isnotnull(s_store_id#68)) (38) BroadcastExchange -Input [2]: [s_store_sk#79, s_store_id#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] +Input [2]: [s_store_sk#67, s_store_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#79] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#67] Join condition: None (40) Project [codegen id : 9] -Output [8]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80] -Input [10]: [d_week_seq#58, ss_store_sk#54, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#79, s_store_id#80] +Output [8]: [d_week_seq#5, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#68] +Input [10]: [d_week_seq#5, ss_store_sk#1, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#67, s_store_id#68] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#82, d_week_seq#83] +Output [2]: [d_month_seq#70, d_week_seq#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1197), LessThanOrEqual(d_month_seq,1208), IsNotNull(d_week_seq)] ReadSchema: struct (42) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#82, d_week_seq#83] +Input [2]: [d_month_seq#70, d_week_seq#71] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#82, d_week_seq#83] -Condition : (((isnotnull(d_month_seq#82) AND (d_month_seq#82 >= 1197)) AND (d_month_seq#82 <= 1208)) AND isnotnull(d_week_seq#83)) +Input [2]: [d_month_seq#70, d_week_seq#71] +Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1197)) AND (d_month_seq#70 <= 1208)) AND isnotnull(d_week_seq#71)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#83] -Input [2]: [d_month_seq#82, d_week_seq#83] +Output [1]: [d_week_seq#71] +Input [2]: [d_month_seq#70, d_week_seq#71] (45) BroadcastExchange -Input [1]: [d_week_seq#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] +Input [1]: [d_week_seq#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#58] -Right keys [1]: [d_week_seq#83] +Left keys [1]: [d_week_seq#5] +Right keys [1]: [d_week_seq#71] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#58 AS d_week_seq2#85, s_store_id#80 AS s_store_id2#86, sun_sales#30 AS sun_sales2#87, mon_sales#31 AS mon_sales2#88, wed_sales#33 AS wed_sales2#89, thu_sales#34 AS thu_sales2#90, fri_sales#35 AS fri_sales2#91, sat_sales#36 AS sat_sales2#92] -Input [9]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80, d_week_seq#83] +Output [8]: [d_week_seq#5 AS d_week_seq2#73, s_store_id#68 AS s_store_id2#74, sun_sales#30 AS sun_sales2#75, mon_sales#31 AS mon_sales2#76, wed_sales#33 AS wed_sales2#77, thu_sales#34 AS thu_sales2#78, fri_sales#35 AS fri_sales2#79, sat_sales#36 AS sat_sales2#80] +Input [9]: [d_week_seq#5, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#68, d_week_seq#71] (48) BroadcastExchange -Input [8]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#93] +Input [8]: [d_week_seq2#73, s_store_id2#74, sun_sales2#75, mon_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#81] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] +Right keys [2]: [s_store_id2#74, (d_week_seq2#73 - 52)] Join condition: None (50) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#87)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#88)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#89)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#90)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#91)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#92)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] -Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#75)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#76)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#84, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#77)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#85, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#78)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#86, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#79)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#87, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#80)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#88] +Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#73, s_store_id2#74, sun_sales2#75, mon_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] -Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] +Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt index d20f7cf4a2857..d0259db7280d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt @@ -171,122 +171,122 @@ Output [10]: [s_store_name#39 AS s_store_name1#44, d_week_seq#5 AS d_week_seq1#4 Input [11]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, s_store_name#39, d_week_seq#42] (26) Scan parquet default.store_sales -Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#56)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] -Condition : isnotnull(ss_store_sk#54) +Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] +Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#56] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#4] Join condition: None (31) Project [codegen id : 6] -Output [4]: [ss_store_sk#54, ss_sales_price#55, d_week_seq#58, d_day_name#59] -Input [6]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56, d_date_sk#57, d_week_seq#58, d_day_name#59] +Output [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] (32) HashAggregate [codegen id : 6] -Input [4]: [ss_store_sk#54, ss_sales_price#55, d_week_seq#58, d_day_name#59] -Keys [2]: [d_week_seq#58, ss_store_sk#54] -Functions [6]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))] -Aggregate Attributes [6]: [sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] -Results [8]: [d_week_seq#58, ss_store_sk#54, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] +Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [6]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))] +Aggregate Attributes [6]: [sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Results [8]: [d_week_seq#5, ss_store_sk#1, sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] (33) Exchange -Input [8]: [d_week_seq#58, ss_store_sk#54, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] -Arguments: hashpartitioning(d_week_seq#58, ss_store_sk#54, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [8]: [d_week_seq#5, ss_store_sk#1, sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [id=#66] (34) HashAggregate [codegen id : 9] -Input [8]: [d_week_seq#58, ss_store_sk#54, sum#66, sum#67, sum#68, sum#69, sum#70, sum#71] -Keys [2]: [d_week_seq#58, ss_store_sk#54] -Functions [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))] -Aggregate Attributes [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END))#73, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END))#74, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END))#75, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END))#76, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END))#77, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))#78] -Results [8]: [d_week_seq#58, ss_store_sk#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#55 ELSE null END))#73,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#55 ELSE null END))#74,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#55 ELSE null END))#75,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#55 ELSE null END))#76,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#55 ELSE null END))#77,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#55 ELSE null END))#78,17,2) AS sat_sales#36] +Input [8]: [d_week_seq#5, ss_store_sk#1, sum#60, sum#61, sum#62, sum#63, sum#64, sum#65] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))] +Aggregate Attributes [6]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29] +Results [8]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29,17,2) AS sat_sales#36] (35) Scan parquet default.store -Output [2]: [s_store_sk#79, s_store_id#80] +Output [2]: [s_store_sk#67, s_store_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (36) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#79, s_store_id#80] +Input [2]: [s_store_sk#67, s_store_id#68] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#79, s_store_id#80] -Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) +Input [2]: [s_store_sk#67, s_store_id#68] +Condition : (isnotnull(s_store_sk#67) AND isnotnull(s_store_id#68)) (38) BroadcastExchange -Input [2]: [s_store_sk#79, s_store_id#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] +Input [2]: [s_store_sk#67, s_store_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#79] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#67] Join condition: None (40) Project [codegen id : 9] -Output [8]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80] -Input [10]: [d_week_seq#58, ss_store_sk#54, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#79, s_store_id#80] +Output [8]: [d_week_seq#5, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#68] +Input [10]: [d_week_seq#5, ss_store_sk#1, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#67, s_store_id#68] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#82, d_week_seq#83] +Output [2]: [d_month_seq#70, d_week_seq#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1197), LessThanOrEqual(d_month_seq,1208), IsNotNull(d_week_seq)] ReadSchema: struct (42) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#82, d_week_seq#83] +Input [2]: [d_month_seq#70, d_week_seq#71] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#82, d_week_seq#83] -Condition : (((isnotnull(d_month_seq#82) AND (d_month_seq#82 >= 1197)) AND (d_month_seq#82 <= 1208)) AND isnotnull(d_week_seq#83)) +Input [2]: [d_month_seq#70, d_week_seq#71] +Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1197)) AND (d_month_seq#70 <= 1208)) AND isnotnull(d_week_seq#71)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#83] -Input [2]: [d_month_seq#82, d_week_seq#83] +Output [1]: [d_week_seq#71] +Input [2]: [d_month_seq#70, d_week_seq#71] (45) BroadcastExchange -Input [1]: [d_week_seq#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] +Input [1]: [d_week_seq#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#58] -Right keys [1]: [d_week_seq#83] +Left keys [1]: [d_week_seq#5] +Right keys [1]: [d_week_seq#71] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#58 AS d_week_seq2#85, s_store_id#80 AS s_store_id2#86, sun_sales#30 AS sun_sales2#87, mon_sales#31 AS mon_sales2#88, wed_sales#33 AS wed_sales2#89, thu_sales#34 AS thu_sales2#90, fri_sales#35 AS fri_sales2#91, sat_sales#36 AS sat_sales2#92] -Input [9]: [d_week_seq#58, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#80, d_week_seq#83] +Output [8]: [d_week_seq#5 AS d_week_seq2#73, s_store_id#68 AS s_store_id2#74, sun_sales#30 AS sun_sales2#75, mon_sales#31 AS mon_sales2#76, wed_sales#33 AS wed_sales2#77, thu_sales#34 AS thu_sales2#78, fri_sales#35 AS fri_sales2#79, sat_sales#36 AS sat_sales2#80] +Input [9]: [d_week_seq#5, sun_sales#30, mon_sales#31, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#68, d_week_seq#71] (48) BroadcastExchange -Input [8]: [d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#93] +Input [8]: [d_week_seq2#73, s_store_id2#74, sun_sales2#75, mon_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#81] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 52)] +Right keys [2]: [s_store_id2#74, (d_week_seq2#73 - 52)] Join condition: None (50) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#87)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#94, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#88)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#95, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#96, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#89)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#97, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#90)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#98, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#91)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#99, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#92)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#100] -Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#85, s_store_id2#86, sun_sales2#87, mon_sales2#88, wed_sales2#89, thu_sales2#90, fri_sales2#91, sat_sales2#92] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#75)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#76)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#84, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#77)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#85, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#78)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#86, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#79)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#87, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#80)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#88] +Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#73, s_store_id2#74, sun_sales2#75, mon_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] -Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#94, (mon_sales1 / mon_sales2)#95, (tue_sales1 / tue_sales1)#96, (wed_sales1 / wed_sales2)#97, (thu_sales1 / thu_sales2)#98, (fri_sales1 / fri_sales2)#99, (sat_sales1 / sat_sales2)#100] +Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt index f4fcd285bb57c..f071af103792d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt @@ -94,158 +94,158 @@ Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) (11) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] (13) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] -Condition : isnotnull(sr_store_sk#15) +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Condition : isnotnull(sr_store_sk#2) (14) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#6] (15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (16) Project [codegen id : 4] -Output [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#18] +Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] (17) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +Keys [2]: [sr_customer_sk#1, sr_store_sk#2] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] +Aggregate Attributes [1]: [sum#14] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] (18) Exchange -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#16] (19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#22] -Results [2]: [sr_store_sk#15 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#22,17,2) AS ctr_total_return#13] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +Keys [2]: [sr_customer_sk#1, sr_store_sk#2] +Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#10] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#10,17,2) AS ctr_total_return#13] (20) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#12, ctr_total_return#13] Keys [1]: [ctr_store_sk#12] Functions [1]: [partial_avg(ctr_total_return#13)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_store_sk#12, sum#25, count#26] +Aggregate Attributes [2]: [sum#17, count#18] +Results [3]: [ctr_store_sk#12, sum#19, count#20] (21) Exchange -Input [3]: [ctr_store_sk#12, sum#25, count#26] -Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_store_sk#12, sum#19, count#20] +Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#21] (22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#12, sum#25, count#26] +Input [3]: [ctr_store_sk#12, sum#19, count#20] Keys [1]: [ctr_store_sk#12] Functions [1]: [avg(ctr_total_return#13)] -Aggregate Attributes [1]: [avg(ctr_total_return#13)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12 AS ctr_store_sk#12#30] +Aggregate Attributes [1]: [avg(ctr_total_return#13)#22] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#22) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12 AS ctr_store_sk#12#24] (23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) (24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#25] (25) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [ctr_store_sk#12#30] -Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_store_sk#12#24] +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) (26) Project [codegen id : 8] Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] -Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12#24] (27) Scan parquet default.store -Output [2]: [s_store_sk#32, s_state#33] +Output [2]: [s_store_sk#26, s_state#27] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] +Input [2]: [s_store_sk#26, s_state#27] (29) Filter [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] -Condition : ((isnotnull(s_state#33) AND (s_state#33 = TN)) AND isnotnull(s_store_sk#32)) +Input [2]: [s_store_sk#26, s_state#27] +Condition : ((isnotnull(s_state#27) AND (s_state#27 = TN)) AND isnotnull(s_store_sk#26)) (30) Project [codegen id : 7] -Output [1]: [s_store_sk#32] -Input [2]: [s_store_sk#32, s_state#33] +Output [1]: [s_store_sk#26] +Input [2]: [s_store_sk#26, s_state#27] (31) BroadcastExchange -Input [1]: [s_store_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [1]: [s_store_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#26] Join condition: None (33) Project [codegen id : 8] Output [1]: [ctr_customer_sk#11] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#32] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#26] (34) Exchange Input [1]: [ctr_customer_sk#11] -Arguments: hashpartitioning(ctr_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#35] +Arguments: hashpartitioning(ctr_customer_sk#11, 5), ENSURE_REQUIREMENTS, [id=#29] (35) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#11] Arguments: [ctr_customer_sk#11 ASC NULLS FIRST], false, 0 (36) Scan parquet default.customer -Output [2]: [c_customer_sk#36, c_customer_id#37] +Output [2]: [c_customer_sk#30, c_customer_id#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#36, c_customer_id#37] +Input [2]: [c_customer_sk#30, c_customer_id#31] (38) Filter [codegen id : 10] -Input [2]: [c_customer_sk#36, c_customer_id#37] -Condition : isnotnull(c_customer_sk#36) +Input [2]: [c_customer_sk#30, c_customer_id#31] +Condition : isnotnull(c_customer_sk#30) (39) Exchange -Input [2]: [c_customer_sk#36, c_customer_id#37] -Arguments: hashpartitioning(c_customer_sk#36, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [2]: [c_customer_sk#30, c_customer_id#31] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (40) Sort [codegen id : 11] -Input [2]: [c_customer_sk#36, c_customer_id#37] -Arguments: [c_customer_sk#36 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#30, c_customer_id#31] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (41) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#36] +Right keys [1]: [c_customer_sk#30] Join condition: None (42) Project [codegen id : 12] -Output [1]: [c_customer_id#37] -Input [3]: [ctr_customer_sk#11, c_customer_sk#36, c_customer_id#37] +Output [1]: [c_customer_id#31] +Input [3]: [ctr_customer_sk#11, c_customer_sk#30, c_customer_id#31] (43) TakeOrderedAndProject -Input [1]: [c_customer_id#37] -Arguments: 100, [c_customer_id#37 ASC NULLS FIRST], [c_customer_id#37] +Input [1]: [c_customer_id#31] +Arguments: 100, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31] ===== Subqueries ===== @@ -258,27 +258,27 @@ BroadcastExchange (48) (44) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#39] +Output [2]: [d_date_sk#6, d_year#33] 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 (45) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#39] +Input [2]: [d_date_sk#6, d_year#33] (46) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2000)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_sk#6)) (47) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#39] +Input [2]: [d_date_sk#6, d_year#33] (48) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index 5066d5c04fa50..33d072fb94143 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -91,146 +91,146 @@ Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] Condition : isnotnull(ctr_total_return#13) (11) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] (13) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] -Condition : isnotnull(sr_store_sk#15) +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Condition : isnotnull(sr_store_sk#2) (14) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#6] (15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (16) Project [codegen id : 4] -Output [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#18] +Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] (17) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +Keys [2]: [sr_customer_sk#1, sr_store_sk#2] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] +Aggregate Attributes [1]: [sum#14] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] (18) Exchange -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#16] (19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#20] -Keys [2]: [sr_customer_sk#14, sr_store_sk#15] -Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#22] -Results [2]: [sr_store_sk#15 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#22,17,2) AS ctr_total_return#13] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +Keys [2]: [sr_customer_sk#1, sr_store_sk#2] +Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#10] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#10,17,2) AS ctr_total_return#13] (20) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#12, ctr_total_return#13] Keys [1]: [ctr_store_sk#12] Functions [1]: [partial_avg(ctr_total_return#13)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_store_sk#12, sum#25, count#26] +Aggregate Attributes [2]: [sum#17, count#18] +Results [3]: [ctr_store_sk#12, sum#19, count#20] (21) Exchange -Input [3]: [ctr_store_sk#12, sum#25, count#26] -Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_store_sk#12, sum#19, count#20] +Arguments: hashpartitioning(ctr_store_sk#12, 5), ENSURE_REQUIREMENTS, [id=#21] (22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#12, sum#25, count#26] +Input [3]: [ctr_store_sk#12, sum#19, count#20] Keys [1]: [ctr_store_sk#12] Functions [1]: [avg(ctr_total_return#13)] -Aggregate Attributes [1]: [avg(ctr_total_return#13)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12 AS ctr_store_sk#12#30] +Aggregate Attributes [1]: [avg(ctr_total_return#13)#22] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#13)#22) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12 AS ctr_store_sk#12#24] (23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) (24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#25] (25) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [ctr_store_sk#12#30] -Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_store_sk#12#24] +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) (26) Project [codegen id : 9] Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] -Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#12#30] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#12#24] (27) Scan parquet default.store -Output [2]: [s_store_sk#32, s_state#33] +Output [2]: [s_store_sk#26, s_state#27] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] +Input [2]: [s_store_sk#26, s_state#27] (29) Filter [codegen id : 7] -Input [2]: [s_store_sk#32, s_state#33] -Condition : ((isnotnull(s_state#33) AND (s_state#33 = TN)) AND isnotnull(s_store_sk#32)) +Input [2]: [s_store_sk#26, s_state#27] +Condition : ((isnotnull(s_state#27) AND (s_state#27 = TN)) AND isnotnull(s_store_sk#26)) (30) Project [codegen id : 7] -Output [1]: [s_store_sk#32] -Input [2]: [s_store_sk#32, s_state#33] +Output [1]: [s_store_sk#26] +Input [2]: [s_store_sk#26, s_state#27] (31) BroadcastExchange -Input [1]: [s_store_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [1]: [s_store_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#12] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#26] Join condition: None (33) Project [codegen id : 9] Output [1]: [ctr_customer_sk#11] -Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#32] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#26] (34) Scan parquet default.customer -Output [2]: [c_customer_sk#35, c_customer_id#36] +Output [2]: [c_customer_sk#29, c_customer_id#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#35, c_customer_id#36] +Input [2]: [c_customer_sk#29, c_customer_id#30] (36) Filter [codegen id : 8] -Input [2]: [c_customer_sk#35, c_customer_id#36] -Condition : isnotnull(c_customer_sk#35) +Input [2]: [c_customer_sk#29, c_customer_id#30] +Condition : isnotnull(c_customer_sk#29) (37) BroadcastExchange -Input [2]: [c_customer_sk#35, c_customer_id#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [2]: [c_customer_sk#29, c_customer_id#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (38) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#11] -Right keys [1]: [c_customer_sk#35] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 9] -Output [1]: [c_customer_id#36] -Input [3]: [ctr_customer_sk#11, c_customer_sk#35, c_customer_id#36] +Output [1]: [c_customer_id#30] +Input [3]: [ctr_customer_sk#11, c_customer_sk#29, c_customer_id#30] (40) TakeOrderedAndProject -Input [1]: [c_customer_id#36] -Arguments: 100, [c_customer_id#36 ASC NULLS FIRST], [c_customer_id#36] +Input [1]: [c_customer_id#30] +Arguments: 100, [c_customer_id#30 ASC NULLS FIRST], [c_customer_id#30] ===== Subqueries ===== @@ -243,27 +243,27 @@ BroadcastExchange (45) (41) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#38] +Output [2]: [d_date_sk#6, d_year#32] 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 (42) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#38] +Input [2]: [d_date_sk#6, d_year#32] (43) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#6)) (44) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#38] +Input [2]: [d_date_sk#6, d_year#32] (45) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index bbf7196929d52..856a8e35d72c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -243,205 +243,205 @@ Arguments: hashpartitioning(c_customer_id#34, c_first_name#35, c_last_name#36, d Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#31, c_preferred_cust_flag#37, c_birth_country#38, c_login#39, c_email_address#40, sum#42] Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#31, c_preferred_cust_flag#37, c_birth_country#38, c_login#39, c_email_address#40] Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#44] -Results [3]: [c_customer_id#34 AS customer_id#45, c_preferred_cust_flag#37 AS customer_preferred_cust_flag#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#47] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#21] +Results [3]: [c_customer_id#34 AS customer_id#44, c_preferred_cust_flag#37 AS customer_preferred_cust_flag#45, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#21,18,2) AS year_total#46] (37) Exchange -Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] -Arguments: hashpartitioning(customer_id#45, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [customer_id#44, customer_preferred_cust_flag#45, year_total#46] +Arguments: hashpartitioning(customer_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (38) Sort [codegen id : 16] -Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] -Arguments: [customer_id#45 ASC NULLS FIRST], false, 0 +Input [3]: [customer_id#44, customer_preferred_cust_flag#45, year_total#46] +Arguments: [customer_id#44 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Right keys [1]: [customer_id#44] Join condition: None (40) Project [codegen id : 17] -Output [4]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47] -Input [5]: [customer_id#22, year_total#23, customer_id#45, customer_preferred_cust_flag#46, year_total#47] +Output [4]: [customer_id#22, year_total#23, customer_preferred_cust_flag#45, year_total#46] +Input [5]: [customer_id#22, year_total#23, customer_id#44, customer_preferred_cust_flag#45, year_total#46] (41) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] (43) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_customer_sk#49) +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_customer_sk#48) (44) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#53, d_year#54] +Output [2]: [d_date_sk#52, d_year#53] (45) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#53] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#52] Join condition: None (46) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54] -Input [6]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54] +Output [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#53] +Input [6]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, ws_sold_date_sk#51, d_date_sk#52, d_year#53] (47) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#53] +Arguments: hashpartitioning(ws_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] (48) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#53] +Arguments: [ws_bill_customer_sk#48 ASC NULLS FIRST], false, 0 (49) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] +Output [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] (50) Sort [codegen id : 22] -Input [8]: [c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] -Arguments: [c_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] +Arguments: [c_customer_sk#55 ASC NULLS FIRST], false, 0 (51) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#56] +Left keys [1]: [ws_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#55] Join condition: None (52) Project [codegen id : 23] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54] -Input [12]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54, c_customer_sk#56, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63] +Output [10]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#53] +Input [12]: [ws_bill_customer_sk#48, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#53, c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] (53) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#64] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#54, sum#65] +Input [10]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, ws_ext_discount_amt#49, ws_ext_list_price#50, d_year#53] +Keys [8]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#49 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#63] +Results [9]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, d_year#53, sum#64] (54) Exchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#54, sum#65] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#54, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [9]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, d_year#53, sum#64] +Arguments: hashpartitioning(c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#65] (55) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#54, sum#65] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#61, c_login#62, c_email_address#63, d_year#54] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2), true)))#67] -Results [2]: [c_customer_id#57 AS customer_id#68, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2), true)))#67,18,2) AS year_total#69] +Input [9]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, d_year#53, sum#64] +Keys [8]: [c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62, d_year#53] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#49 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#49 as decimal(8,2)))), DecimalType(8,2), true)))#66] +Results [2]: [c_customer_id#56 AS customer_id#67, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#49 as decimal(8,2)))), DecimalType(8,2), true)))#66,18,2) AS year_total#68] (56) Filter [codegen id : 24] -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) +Input [2]: [customer_id#67, year_total#68] +Condition : (isnotnull(year_total#68) AND (year_total#68 > 0.00)) (57) Exchange -Input [2]: [customer_id#68, year_total#69] -Arguments: hashpartitioning(customer_id#68, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [2]: [customer_id#67, year_total#68] +Arguments: hashpartitioning(customer_id#67, 5), ENSURE_REQUIREMENTS, [id=#69] (58) Sort [codegen id : 25] -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#67, year_total#68] +Arguments: [customer_id#67 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#68] +Right keys [1]: [customer_id#67] Join condition: None (60) Project [codegen id : 26] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#69] -Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#68, year_total#69] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#45, year_total#46, year_total#68] +Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#45, year_total#46, customer_id#67, year_total#68] (61) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Output [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] (63) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_bill_customer_sk#71) +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73] +Condition : isnotnull(ws_bill_customer_sk#70) (64) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#75, d_year#76] +Output [2]: [d_date_sk#74, d_year#75] (65) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#74] -Right keys [1]: [d_date_sk#75] +Left keys [1]: [ws_sold_date_sk#73] +Right keys [1]: [d_date_sk#74] Join condition: None (66) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#76] -Input [6]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#75, d_year#76] +Output [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Input [6]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, ws_sold_date_sk#73, d_date_sk#74, d_year#75] (67) Exchange -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#76] -Arguments: hashpartitioning(ws_bill_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Arguments: hashpartitioning(ws_bill_customer_sk#70, 5), ENSURE_REQUIREMENTS, [id=#76] (68) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#76] -Arguments: [ws_bill_customer_sk#71 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Arguments: [ws_bill_customer_sk#70 ASC NULLS FIRST], false, 0 (69) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Output [8]: [c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84] (70) Sort [codegen id : 31] -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Arguments: [c_customer_sk#78 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#71] -Right keys [1]: [c_customer_sk#78] +Left keys [1]: [ws_bill_customer_sk#70] +Right keys [1]: [c_customer_sk#77] Join condition: None (72) Project [codegen id : 32] -Output [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#76] -Input [12]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#76, c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Output [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Input [12]: [ws_bill_customer_sk#70, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75, c_customer_sk#77, c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84] (73) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#76] -Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#76] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#73 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#86] -Results [9]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#76, sum#87] +Input [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75] +Keys [8]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#75] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#85] +Results [9]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#75, sum#86] (74) Exchange -Input [9]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#76, sum#87] -Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#76, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [9]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#75, sum#86] +Arguments: hashpartitioning(c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#75, 5), ENSURE_REQUIREMENTS, [id=#87] (75) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#76, sum#87] -Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#76] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#73 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#73 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(8,2)))), DecimalType(8,2), true)))#89] -Results [2]: [c_customer_id#79 AS customer_id#90, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#73 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#72 as decimal(8,2)))), DecimalType(8,2), true)))#89,18,2) AS year_total#91] +Input [9]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#75, sum#86] +Keys [8]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#75] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2), true)))#66] +Results [2]: [c_customer_id#78 AS customer_id#88, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2), true)))#66,18,2) AS year_total#89] (76) Exchange -Input [2]: [customer_id#90, year_total#91] -Arguments: hashpartitioning(customer_id#90, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [2]: [customer_id#88, year_total#89] +Arguments: hashpartitioning(customer_id#88, 5), ENSURE_REQUIREMENTS, [id=#90] (77) Sort [codegen id : 34] -Input [2]: [customer_id#90, year_total#91] -Arguments: [customer_id#90 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#88, year_total#89] +Arguments: [customer_id#88 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#90] -Join condition: (CASE WHEN (year_total#69 > 0.00) THEN CheckOverflow((promote_precision(year_total#91) / promote_precision(year_total#69)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#88] +Join condition: (CASE WHEN (year_total#68 > 0.00) THEN CheckOverflow((promote_precision(year_total#89) / promote_precision(year_total#68)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) (79) Project [codegen id : 35] -Output [1]: [customer_preferred_cust_flag#46] -Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#69, customer_id#90, year_total#91] +Output [1]: [customer_preferred_cust_flag#45] +Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#45, year_total#46, year_total#68, customer_id#88, year_total#89] (80) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#46] -Arguments: 100, [customer_preferred_cust_flag#46 ASC NULLS FIRST], [customer_preferred_cust_flag#46] +Input [1]: [customer_preferred_cust_flag#45] +Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45] ===== Subqueries ===== @@ -468,7 +468,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (84) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#91] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (88) @@ -493,10 +493,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (88) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#73 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index f18f0b69b44fd..889e6c641f05b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -218,199 +218,199 @@ Arguments: hashpartitioning(c_customer_id#24, c_first_name#25, c_last_name#26, d Input [9]: [c_customer_id#24, c_first_name#25, c_last_name#26, d_year#38, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, sum#40] Keys [8]: [c_customer_id#24, c_first_name#25, c_last_name#26, d_year#38, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30] Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#42] -Results [3]: [c_customer_id#24 AS customer_id#43, c_preferred_cust_flag#27 AS customer_preferred_cust_flag#44, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#42,18,2) AS year_total#45] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#20] +Results [3]: [c_customer_id#24 AS customer_id#42, c_preferred_cust_flag#27 AS customer_preferred_cust_flag#43, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#20,18,2) AS year_total#44] (32) BroadcastExchange -Input [3]: [customer_id#43, customer_preferred_cust_flag#44, year_total#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [3]: [customer_id#42, customer_preferred_cust_flag#43, year_total#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] (33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#42] Join condition: None (34) Project [codegen id : 16] -Output [4]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45] -Input [5]: [customer_id#21, year_total#22, customer_id#43, customer_preferred_cust_flag#44, year_total#45] +Output [4]: [customer_id#21, year_total#22, customer_preferred_cust_flag#43, year_total#44] +Input [5]: [customer_id#21, year_total#22, customer_id#42, customer_preferred_cust_flag#43, year_total#44] (35) Scan parquet default.customer -Output [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] +Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] +Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] (37) Filter [codegen id : 10] -Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) +Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +Condition : (isnotnull(c_customer_sk#46) AND isnotnull(c_customer_id#47)) (38) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#55, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58] +Output [4]: [ws_bill_customer_sk#54, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#55, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58] +Input [4]: [ws_bill_customer_sk#54, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57] (40) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#55, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#55) +Input [4]: [ws_bill_customer_sk#54, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57] +Condition : isnotnull(ws_bill_customer_sk#54) (41) BroadcastExchange -Input [4]: [ws_bill_customer_sk#55, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] +Input [4]: [ws_bill_customer_sk#54, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] (42) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#55] +Left keys [1]: [c_customer_sk#46] +Right keys [1]: [ws_bill_customer_sk#54] Join condition: None (43) Project [codegen id : 10] -Output [10]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58] -Input [12]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, ws_bill_customer_sk#55, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58] +Output [10]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57] +Input [12]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, ws_bill_customer_sk#54, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57] (44) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#60, d_year#61] +Output [2]: [d_date_sk#59, d_year#60] (45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#60] +Left keys [1]: [ws_sold_date_sk#57] +Right keys [1]: [d_date_sk#59] Join condition: None (46) Project [codegen id : 10] -Output [10]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, ws_ext_discount_amt#56, ws_ext_list_price#57, d_year#61] -Input [12]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, ws_ext_discount_amt#56, ws_ext_list_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61] +Output [10]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, ws_ext_discount_amt#55, ws_ext_list_price#56, d_year#60] +Input [12]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, ws_ext_discount_amt#55, ws_ext_list_price#56, ws_sold_date_sk#57, d_date_sk#59, d_year#60] (47) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, ws_ext_discount_amt#56, ws_ext_list_price#57, d_year#61] -Keys [8]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#57 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#56 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#62] -Results [9]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#61, sum#63] +Input [10]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, ws_ext_discount_amt#55, ws_ext_list_price#56, d_year#60] +Keys [8]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, d_year#60] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#56 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#55 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#61] +Results [9]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, d_year#60, sum#62] (48) Exchange -Input [9]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#61, sum#63] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [9]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, d_year#60, sum#62] +Arguments: hashpartitioning(c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, d_year#60, 5), ENSURE_REQUIREMENTS, [id=#63] (49) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#61, sum#63] -Keys [8]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#61] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#57 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#56 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#57 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#56 as decimal(8,2)))), DecimalType(8,2), true)))#65] -Results [2]: [c_customer_id#48 AS customer_id#66, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#57 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#56 as decimal(8,2)))), DecimalType(8,2), true)))#65,18,2) AS year_total#67] +Input [9]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, d_year#60, sum#62] +Keys [8]: [c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53, d_year#60] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#56 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#55 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#56 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#55 as decimal(8,2)))), DecimalType(8,2), true)))#64] +Results [2]: [c_customer_id#47 AS customer_id#65, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#56 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#55 as decimal(8,2)))), DecimalType(8,2), true)))#64,18,2) AS year_total#66] (50) Filter [codegen id : 11] -Input [2]: [customer_id#66, year_total#67] -Condition : (isnotnull(year_total#67) AND (year_total#67 > 0.00)) +Input [2]: [customer_id#65, year_total#66] +Condition : (isnotnull(year_total#66) AND (year_total#66 > 0.00)) (51) BroadcastExchange -Input [2]: [customer_id#66, year_total#67] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#68] +Input [2]: [customer_id#65, year_total#66] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#67] (52) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#66] +Right keys [1]: [customer_id#65] Join condition: None (53) Project [codegen id : 16] -Output [5]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, year_total#67] -Input [6]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, customer_id#66, year_total#67] +Output [5]: [customer_id#21, year_total#22, customer_preferred_cust_flag#43, year_total#44, year_total#66] +Input [6]: [customer_id#21, year_total#22, customer_preferred_cust_flag#43, year_total#44, customer_id#65, year_total#66] (54) Scan parquet default.customer -Output [8]: [c_customer_sk#69, c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76] +Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#69, c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76] +Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] (56) Filter [codegen id : 14] -Input [8]: [c_customer_sk#69, c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76] -Condition : (isnotnull(c_customer_sk#69) AND isnotnull(c_customer_id#70)) +Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] +Condition : (isnotnull(c_customer_sk#68) AND isnotnull(c_customer_id#69)) (57) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] +Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] (59) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_bill_customer_sk#77) +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#76) (60) BroadcastExchange -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] +Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#69] -Right keys [1]: [ws_bill_customer_sk#77] +Left keys [1]: [c_customer_sk#68] +Right keys [1]: [ws_bill_customer_sk#76] Join condition: None (62) Project [codegen id : 14] -Output [10]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] -Input [12]: [c_customer_sk#69, c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] +Output [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [12]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] (63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#82, d_year#83] +Output [2]: [d_date_sk#81, d_year#82] (64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#82] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#81] Join condition: None (65) Project [codegen id : 14] -Output [10]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#83] -Input [12]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80, d_date_sk#82, d_year#83] +Output [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] +Input [12]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#81, d_year#82] (66) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#83] -Keys [8]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, d_year#83] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#79 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#78 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#84] -Results [9]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, d_year#83, sum#85] +Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] +Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#82] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#83] +Results [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#82, sum#84] (67) Exchange -Input [9]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, d_year#83, sum#85] -Arguments: hashpartitioning(c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, d_year#83, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#82, sum#84] +Arguments: hashpartitioning(c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#85] (68) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, d_year#83, sum#85] -Keys [8]: [c_customer_id#70, c_first_name#71, c_last_name#72, c_preferred_cust_flag#73, c_birth_country#74, c_login#75, c_email_address#76, d_year#83] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#79 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#78 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#79 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#78 as decimal(8,2)))), DecimalType(8,2), true)))#87] -Results [2]: [c_customer_id#70 AS customer_id#88, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#79 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#78 as decimal(8,2)))), DecimalType(8,2), true)))#87,18,2) AS year_total#89] +Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#82, sum#84] +Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#82] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#64] +Results [2]: [c_customer_id#69 AS customer_id#86, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(8,2)))), DecimalType(8,2), true)))#64,18,2) AS year_total#87] (69) BroadcastExchange -Input [2]: [customer_id#88, year_total#89] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] +Input [2]: [customer_id#86, year_total#87] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#88] (70) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#88] -Join condition: (CASE WHEN (year_total#67 > 0.00) THEN CheckOverflow((promote_precision(year_total#89) / promote_precision(year_total#67)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#45) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#86] +Join condition: (CASE WHEN (year_total#66 > 0.00) THEN CheckOverflow((promote_precision(year_total#87) / promote_precision(year_total#66)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#44) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE null END) (71) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#44] -Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#44, year_total#45, year_total#67, customer_id#88, year_total#89] +Output [1]: [customer_preferred_cust_flag#43] +Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#43, year_total#44, year_total#66, customer_id#86, year_total#87] (72) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#44] -Arguments: 100, [customer_preferred_cust_flag#44 ASC NULLS FIRST], [customer_preferred_cust_flag#44] +Input [1]: [customer_preferred_cust_flag#43] +Arguments: 100, [customer_preferred_cust_flag#43 ASC NULLS FIRST], [customer_preferred_cust_flag#43] ===== Subqueries ===== @@ -437,7 +437,7 @@ Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_ (76) BroadcastExchange Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#89] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (80) @@ -462,10 +462,10 @@ Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_ (80) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index c31bb7470648f..b4e24c7a7fbdd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -145,64 +145,64 @@ Output [8]: [d_week_seq#10 AS d_week_seq1#45, sun_sales#35 AS sun_sales1#46, mon Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#42] (24) ReusedExchange [Reuses operator id: 15] -Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] (25) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] -Keys [1]: [d_week_seq#53] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#4 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#4 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#4 ELSE null END))#67, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#4 ELSE null END))#68] -Results [8]: [d_week_seq#53, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#4 ELSE null END))#62,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#4 ELSE null END))#63,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#4 ELSE null END))#64,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#4 ELSE null END))#65,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#4 ELSE null END))#66,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#4 ELSE null END))#67,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#4 ELSE null END))#68,17,2) AS sat_sales#41] +Input [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [d_week_seq#10] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#34] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#28,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#29,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#30,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#31,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#32,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#33,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#34,17,2) AS sat_sales#41] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#69, d_year#70] +Output [2]: [d_week_seq#60, d_year#61] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#69, d_year#70] +Input [2]: [d_week_seq#60, d_year#61] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#69, d_year#70] -Condition : ((isnotnull(d_year#70) AND (d_year#70 = 2002)) AND isnotnull(d_week_seq#69)) +Input [2]: [d_week_seq#60, d_year#61] +Condition : ((isnotnull(d_year#61) AND (d_year#61 = 2002)) AND isnotnull(d_week_seq#60)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#69] -Input [2]: [d_week_seq#69, d_year#70] +Output [1]: [d_week_seq#60] +Input [2]: [d_week_seq#60, d_year#61] (30) BroadcastExchange -Input [1]: [d_week_seq#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#53] -Right keys [1]: [d_week_seq#69] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#60] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#35 AS sun_sales2#73, mon_sales#36 AS mon_sales2#74, tue_sales#37 AS tue_sales2#75, wed_sales#38 AS wed_sales2#76, thu_sales#39 AS thu_sales2#77, fri_sales#40 AS fri_sales2#78, sat_sales#41 AS sat_sales2#79] -Input [9]: [d_week_seq#53, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#69] +Output [8]: [d_week_seq#10 AS d_week_seq2#63, sun_sales#35 AS sun_sales2#64, mon_sales#36 AS mon_sales2#65, tue_sales#37 AS tue_sales2#66, wed_sales#38 AS wed_sales2#67, thu_sales#39 AS thu_sales2#68, fri_sales#40 AS fri_sales2#69, sat_sales#41 AS sat_sales2#70] +Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#60] (33) BroadcastExchange -Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#80] +Input [8]: [d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#71] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#72 - 53)] +Right keys [1]: [(d_week_seq2#63 - 53)] Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#73)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#74)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#75)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#76)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#77)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#78)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#79)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#64)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#72, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#65)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#73, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#66)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#74, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#67)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#75, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#68)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#76, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#69)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#77, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#70)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#78] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#72, round((mon_sales1 / mon_sales2), 2)#73, round((tue_sales1 / tue_sales2), 2)#74, round((wed_sales1 / wed_sales2), 2)#75, round((thu_sales1 / thu_sales2), 2)#76, round((fri_sales1 / fri_sales2), 2)#77, round((sat_sales1 / sat_sales2), 2)#78] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#79] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#72, round((mon_sales1 / mon_sales2), 2)#73, round((tue_sales1 / tue_sales2), 2)#74, round((wed_sales1 / wed_sales2), 2)#75, round((thu_sales1 / thu_sales2), 2)#76, round((fri_sales1 / fri_sales2), 2)#77, round((sat_sales1 / sat_sales2), 2)#78] Arguments: [d_week_seq1#45 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index c31bb7470648f..b4e24c7a7fbdd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -145,64 +145,64 @@ Output [8]: [d_week_seq#10 AS d_week_seq1#45, sun_sales#35 AS sun_sales1#46, mon Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#42] (24) ReusedExchange [Reuses operator id: 15] -Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] (25) HashAggregate [codegen id : 11] -Input [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] -Keys [1]: [d_week_seq#53] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#4 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#4 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#4 ELSE null END))#67, sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#4 ELSE null END))#68] -Results [8]: [d_week_seq#53, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Sunday ) THEN sales_price#4 ELSE null END))#62,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Monday ) THEN sales_price#4 ELSE null END))#63,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Tuesday ) THEN sales_price#4 ELSE null END))#64,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Wednesday) THEN sales_price#4 ELSE null END))#65,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Thursday ) THEN sales_price#4 ELSE null END))#66,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Friday ) THEN sales_price#4 ELSE null END))#67,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#61 = Saturday ) THEN sales_price#4 ELSE null END))#68,17,2) AS sat_sales#41] +Input [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [d_week_seq#10] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#32, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#33, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#34] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#28,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#29,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#30,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#31,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#32,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#33,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#34,17,2) AS sat_sales#41] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#69, d_year#70] +Output [2]: [d_week_seq#60, d_year#61] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [d_week_seq#69, d_year#70] +Input [2]: [d_week_seq#60, d_year#61] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#69, d_year#70] -Condition : ((isnotnull(d_year#70) AND (d_year#70 = 2002)) AND isnotnull(d_week_seq#69)) +Input [2]: [d_week_seq#60, d_year#61] +Condition : ((isnotnull(d_year#61) AND (d_year#61 = 2002)) AND isnotnull(d_week_seq#60)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#69] -Input [2]: [d_week_seq#69, d_year#70] +Output [1]: [d_week_seq#60] +Input [2]: [d_week_seq#60, d_year#61] (30) BroadcastExchange -Input [1]: [d_week_seq#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#53] -Right keys [1]: [d_week_seq#69] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#60] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#53 AS d_week_seq2#72, sun_sales#35 AS sun_sales2#73, mon_sales#36 AS mon_sales2#74, tue_sales#37 AS tue_sales2#75, wed_sales#38 AS wed_sales2#76, thu_sales#39 AS thu_sales2#77, fri_sales#40 AS fri_sales2#78, sat_sales#41 AS sat_sales2#79] -Input [9]: [d_week_seq#53, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#69] +Output [8]: [d_week_seq#10 AS d_week_seq2#63, sun_sales#35 AS sun_sales2#64, mon_sales#36 AS mon_sales2#65, tue_sales#37 AS tue_sales2#66, wed_sales#38 AS wed_sales2#67, thu_sales#39 AS thu_sales2#68, fri_sales#40 AS fri_sales2#69, sat_sales#41 AS sat_sales2#70] +Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#60] (33) BroadcastExchange -Input [8]: [d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#80] +Input [8]: [d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#71] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#72 - 53)] +Right keys [1]: [(d_week_seq2#63 - 53)] Join condition: None (35) Project [codegen id : 12] -Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#73)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#81, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#74)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#82, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#75)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#83, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#76)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#84, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#77)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#85, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#78)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#86, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#79)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#87] -Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#72, sun_sales2#73, mon_sales2#74, tue_sales2#75, wed_sales2#76, thu_sales2#77, fri_sales2#78, sat_sales2#79] +Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#64)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#72, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#65)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#73, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#66)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#74, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#67)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#75, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#68)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#76, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#69)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#77, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#70)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#78] +Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#72, round((mon_sales1 / mon_sales2), 2)#73, round((tue_sales1 / tue_sales2), 2)#74, round((wed_sales1 / wed_sales2), 2)#75, round((thu_sales1 / thu_sales2), 2)#76, round((fri_sales1 / fri_sales2), 2)#77, round((sat_sales1 / sat_sales2), 2)#78] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#79] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#81, round((mon_sales1 / mon_sales2), 2)#82, round((tue_sales1 / tue_sales2), 2)#83, round((wed_sales1 / wed_sales2), 2)#84, round((thu_sales1 / thu_sales2), 2)#85, round((fri_sales1 / fri_sales2), 2)#86, round((sat_sales1 / sat_sales2), 2)#87] +Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#72, round((mon_sales1 / mon_sales2), 2)#73, round((tue_sales1 / tue_sales2), 2)#74, round((wed_sales1 / wed_sales2), 2)#75, round((thu_sales1 / thu_sales2), 2)#76, round((fri_sales1 / fri_sales2), 2)#77, round((sat_sales1 / sat_sales2), 2)#78] Arguments: [d_week_seq1#45 ASC NULLS FIRST], true, 0 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 74092884ec79b..be706fee66776 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 @@ -341,41 +341,41 @@ Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price Arguments: [ws_item_sk#41 ASC NULLS FIRST], false, 0 (56) ReusedExchange [Reuses operator id: 11] -Output [2]: [ss_item_sk#47, d_date#48] +Output [2]: [ss_item_sk#8, d_date#12] (57) Sort [codegen id : 22] -Input [2]: [ss_item_sk#47, d_date#48] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#8, d_date#12] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 (58) ReusedExchange [Reuses operator id: 16] -Output [2]: [i_item_sk#49, i_item_desc#50] +Output [2]: [i_item_sk#14, i_item_desc#15] (59) Sort [codegen id : 24] -Input [2]: [i_item_sk#49, i_item_desc#50] -Arguments: [i_item_sk#49 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#14, i_item_desc#15] +Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 (60) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#47] -Right keys [1]: [i_item_sk#49] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#14] Join condition: None (61) Project [codegen id : 25] -Output [3]: [d_date#48, i_item_sk#49, substr(i_item_desc#50, 1, 30) AS _groupingexpression#51] -Input [4]: [ss_item_sk#47, d_date#48, i_item_sk#49, i_item_desc#50] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#47] +Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] (62) HashAggregate [codegen id : 25] -Input [3]: [d_date#48, i_item_sk#49, _groupingexpression#51] -Keys [3]: [_groupingexpression#51, i_item_sk#49, d_date#48] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#47] +Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#52] -Results [4]: [_groupingexpression#51, i_item_sk#49, d_date#48, count#53] +Aggregate Attributes [1]: [count#18] +Results [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] (63) HashAggregate [codegen id : 25] -Input [4]: [_groupingexpression#51, i_item_sk#49, d_date#48, count#53] -Keys [3]: [_groupingexpression#51, i_item_sk#49, d_date#48] +Input [4]: [_groupingexpression#47, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#47, i_item_sk#14, d_date#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#54] -Results [2]: [i_item_sk#49 AS item_sk#21, count(1)#54 AS cnt#22] +Aggregate Attributes [1]: [count(1)#20] +Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] (64) Filter [codegen id : 25] Input [2]: [item_sk#21, cnt#22] @@ -400,64 +400,64 @@ Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price (69) Exchange Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#55] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#48] (70) Sort [codegen id : 27] Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (71) ReusedExchange [Reuses operator id: 33] -Output [3]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] (72) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58] -Arguments: [ss_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 (73) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#59] +Output [1]: [c_customer_sk#29] (74) Sort [codegen id : 31] -Input [1]: [c_customer_sk#59] -Arguments: [c_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#56] -Right keys [1]: [c_customer_sk#59] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#29] Join condition: None (76) Project [codegen id : 32] -Output [3]: [ss_quantity#57, ss_sales_price#58, c_customer_sk#59] -Input [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, c_customer_sk#59] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] (77) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#57, ss_sales_price#58, c_customer_sk#59] -Keys [1]: [c_customer_sk#59] -Functions [1]: [partial_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 [2]: [sum#60, isEmpty#61] -Results [3]: [c_customer_sk#59, sum#62, isEmpty#63] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (78) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#59, sum#62, isEmpty#63] -Keys [1]: [c_customer_sk#59] -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))#64] -Results [2]: [c_customer_sk#59, 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))#64 AS ssales#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 AS ssales#36] (79) Filter [codegen id : 32] -Input [2]: [c_customer_sk#59, ssales#36] +Input [2]: [c_customer_sk#29, ssales#36] Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) (80) Project [codegen id : 32] -Output [1]: [c_customer_sk#59] -Input [2]: [c_customer_sk#59, ssales#36] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, ssales#36] (81) Sort [codegen id : 32] -Input [1]: [c_customer_sk#59] -Arguments: [c_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (82) SortMergeJoin [codegen id : 34] Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#59] +Right keys [1]: [c_customer_sk#29] Join condition: None (83) Project [codegen id : 34] @@ -465,16 +465,16 @@ Output [3]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] (84) ReusedExchange [Reuses operator id: 95] -Output [1]: [d_date_sk#65] +Output [1]: [d_date_sk#49] (85) BroadcastHashJoin [codegen id : 34] Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#65] +Right keys [1]: [d_date_sk#49] Join condition: None (86) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#66] -Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#65] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#50] +Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#49] (87) Union @@ -482,19 +482,19 @@ Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#65] Input [1]: [sales#40] Keys: [] Functions [1]: [partial_sum(sales#40)] -Aggregate Attributes [2]: [sum#67, isEmpty#68] -Results [2]: [sum#69, isEmpty#70] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [2]: [sum#53, isEmpty#54] (89) Exchange -Input [2]: [sum#69, isEmpty#70] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [sum#53, isEmpty#54] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#55] (90) HashAggregate [codegen id : 36] -Input [2]: [sum#69, isEmpty#70] +Input [2]: [sum#53, isEmpty#54] Keys: [] Functions [1]: [sum(sales#40)] -Aggregate Attributes [1]: [sum(sales#40)#72] -Results [1]: [sum(sales#40)#72 AS sum(sales)#73] +Aggregate Attributes [1]: [sum(sales#40)#56] +Results [1]: [sum(sales#40)#56 AS sum(sales)#57] ===== Subqueries ===== @@ -507,26 +507,26 @@ BroadcastExchange (95) (91) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#74, d_moy#75] +Output [3]: [d_date_sk#39, d_year#58, d_moy#59] 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 (92) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#74, d_moy#75] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] (93) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#74, d_moy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_moy#75)) AND (d_year#74 = 2000)) AND (d_moy#75 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] +Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#39)) (94) Project [codegen id : 1] Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#74, d_moy#75] +Input [3]: [d_date_sk#39, d_year#58, d_moy#59] (95) BroadcastExchange Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (100) @@ -537,26 +537,26 @@ BroadcastExchange (100) (96) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#77] +Output [3]: [d_date_sk#11, d_date#12, d_year#61] 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 (97) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#77] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] (98) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#77] -Condition : (d_year#77 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_date#12, d_year#61] +Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (99) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#77] +Input [3]: [d_date_sk#11, d_date#12, d_year#61] (100) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#62] Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#37, [id=#38] * HashAggregate (117) @@ -579,89 +579,89 @@ Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82] +Output [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#82), dynamicpruningexpression(ss_sold_date_sk#82 IN dynamicpruning#83)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, ss_sold_date_sk#82] -Condition : isnotnull(ss_customer_sk#79) +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_customer_sk#63) (104) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#84] +Output [1]: [d_date_sk#68] (105) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#82] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#68] Join condition: None (106) Project [codegen id : 2] -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] +Output [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Input [5]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, d_date_sk#68] (107) Exchange -Input [3]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81] -Arguments: hashpartitioning(ss_customer_sk#79, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Arguments: hashpartitioning(ss_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] (108) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81] -Arguments: [ss_customer_sk#79 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65] +Arguments: [ss_customer_sk#63 ASC NULLS FIRST], false, 0 (109) ReusedExchange [Reuses operator id: 38] -Output [1]: [c_customer_sk#86] +Output [1]: [c_customer_sk#70] (110) Sort [codegen id : 5] -Input [1]: [c_customer_sk#86] -Arguments: [c_customer_sk#86 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#70] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 (111) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#79] -Right keys [1]: [c_customer_sk#86] +Left keys [1]: [ss_customer_sk#63] +Right keys [1]: [c_customer_sk#70] Join condition: None (112) Project [codegen id : 6] -Output [3]: [ss_quantity#80, ss_sales_price#81, c_customer_sk#86] -Input [4]: [ss_customer_sk#79, ss_quantity#80, ss_sales_price#81, c_customer_sk#86] +Output [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] +Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, c_customer_sk#70] (113) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#80, ss_sales_price#81, c_customer_sk#86] -Keys [1]: [c_customer_sk#86] -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#87, isEmpty#88] -Results [3]: [c_customer_sk#86, sum#89, isEmpty#90] +Input [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#70] +Keys [1]: [c_customer_sk#70] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#71, isEmpty#72] +Results [3]: [c_customer_sk#70, sum#73, isEmpty#74] (114) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#86, sum#89, isEmpty#90] -Keys [1]: [c_customer_sk#86] -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))#91] -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))#91 AS csales#92] +Input [3]: [c_customer_sk#70, sum#73, isEmpty#74] +Keys [1]: [c_customer_sk#70] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2), true))#75 AS csales#76] (115) HashAggregate [codegen id : 6] -Input [1]: [csales#92] +Input [1]: [csales#76] Keys: [] -Functions [1]: [partial_max(csales#92)] -Aggregate Attributes [1]: [max#93] -Results [1]: [max#94] +Functions [1]: [partial_max(csales#76)] +Aggregate Attributes [1]: [max#77] +Results [1]: [max#78] (116) Exchange -Input [1]: [max#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [1]: [max#78] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#79] (117) HashAggregate [codegen id : 7] -Input [1]: [max#94] +Input [1]: [max#78] Keys: [] -Functions [1]: [max(csales#92)] -Aggregate Attributes [1]: [max(csales#92)#96] -Results [1]: [max(csales#92)#96 AS tpcds_cmax#97] +Functions [1]: [max(csales#76)] +Aggregate Attributes [1]: [max(csales#76)#80] +Results [1]: [max(csales#76)#80 AS tpcds_cmax#81] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#82 IN dynamicpruning#83 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 BroadcastExchange (122) +- * Project (121) +- * Filter (120) @@ -670,26 +670,26 @@ BroadcastExchange (122) (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#84, d_year#98] +Output [2]: [d_date_sk#68, d_year#82] 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 (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#84, d_year#98] +Input [2]: [d_date_sk#68, d_year#82] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#84, d_year#98] -Condition : (d_year#98 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#84)) +Input [2]: [d_date_sk#68, d_year#82] +Condition : (d_year#82 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#84] -Input [2]: [d_date_sk#84, d_year#98] +Output [1]: [d_date_sk#68] +Input [2]: [d_date_sk#68, d_year#82] (122) BroadcastExchange -Input [1]: [d_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#99] +Input [1]: [d_date_sk#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 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 fc55be7f24cd8..1de23e1f4d2ab 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 @@ -305,30 +305,30 @@ Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_da Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (53) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#47, sum#48, isEmpty#49] +Output [3]: [c_customer_sk#28, sum#32, isEmpty#33] (54) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#47, sum#48, isEmpty#49] -Keys [1]: [c_customer_sk#47] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#51 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#51 as decimal(12,2)))), DecimalType(18,2), true))#52] -Results [2]: [c_customer_sk#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#51 as decimal(12,2)))), DecimalType(18,2), true))#52 AS ssales#36] +Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 AS ssales#36] (55) Filter [codegen id : 20] -Input [2]: [c_customer_sk#47, ssales#36] +Input [2]: [c_customer_sk#28, ssales#36] Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) (56) Project [codegen id : 20] -Output [1]: [c_customer_sk#47] -Input [2]: [c_customer_sk#47, ssales#36] +Output [1]: [c_customer_sk#28] +Input [2]: [c_customer_sk#28, ssales#36] (57) Sort [codegen id : 20] -Input [1]: [c_customer_sk#47] -Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (58) SortMergeJoin [codegen id : 22] Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#28] Join condition: None (59) Project [codegen id : 22] @@ -336,16 +336,16 @@ Output [3]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] (60) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#53] +Output [1]: [d_date_sk#47] (61) BroadcastHashJoin [codegen id : 22] Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#53] +Right keys [1]: [d_date_sk#47] Join condition: None (62) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#54] -Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#53] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#43 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#44 as decimal(12,2)))), DecimalType(18,2), true) AS sales#48] +Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#47] (63) Union @@ -353,19 +353,19 @@ Input [4]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, d_date_sk#53] Input [1]: [sales#40] Keys: [] Functions [1]: [partial_sum(sales#40)] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [2]: [sum#57, isEmpty#58] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [2]: [sum#51, isEmpty#52] (65) Exchange -Input [2]: [sum#57, isEmpty#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#53] (66) HashAggregate [codegen id : 24] -Input [2]: [sum#57, isEmpty#58] +Input [2]: [sum#51, isEmpty#52] Keys: [] Functions [1]: [sum(sales#40)] -Aggregate Attributes [1]: [sum(sales#40)#60] -Results [1]: [sum(sales#40)#60 AS sum(sales)#61] +Aggregate Attributes [1]: [sum(sales#40)#54] +Results [1]: [sum(sales#40)#54 AS sum(sales)#55] ===== Subqueries ===== @@ -378,26 +378,26 @@ BroadcastExchange (71) (67) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#62, d_moy#63] +Output [3]: [d_date_sk#39, d_year#56, d_moy#57] 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 (68) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#62, d_moy#63] +Input [3]: [d_date_sk#39, d_year#56, d_moy#57] (69) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#62, d_moy#63] -Condition : ((((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 2000)) AND (d_moy#63 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#39, d_year#56, d_moy#57] +Condition : ((((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 2000)) AND (d_moy#57 = 2)) AND isnotnull(d_date_sk#39)) (70) Project [codegen id : 1] Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#62, d_moy#63] +Input [3]: [d_date_sk#39, d_year#56, d_moy#57] (71) BroadcastExchange Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) @@ -408,26 +408,26 @@ BroadcastExchange (76) (72) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#65] +Output [3]: [d_date_sk#10, d_date#11, d_year#59] 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 (73) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#65] +Input [3]: [d_date_sk#10, d_date#11, d_year#59] (74) Filter [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#65] -Condition : (d_year#65 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_date#11, d_year#59] +Condition : (d_year#59 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (75) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] -Input [3]: [d_date_sk#10, d_date#11, d_year#65] +Input [3]: [d_date_sk#10, d_date#11, d_year#59] (76) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#37, [id=#38] * HashAggregate (91) @@ -448,81 +448,81 @@ Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] +Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_sold_date_sk#64 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] +Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#67, ss_quantity#68, ss_sales_price#69, ss_sold_date_sk#70] -Condition : isnotnull(ss_customer_sk#67) +Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] +Condition : isnotnull(ss_customer_sk#61) (80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#72] +Output [1]: [c_customer_sk#66] (81) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#67] -Right keys [1]: [c_customer_sk#72] +Left keys [1]: [ss_customer_sk#61] +Right keys [1]: [c_customer_sk#66] Join condition: None (82) Project [codegen id : 3] -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] +Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] +Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] (83) ReusedExchange [Reuses operator id: 96] -Output [1]: [d_date_sk#73] +Output [1]: [d_date_sk#67] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#70] -Right keys [1]: [d_date_sk#73] +Left keys [1]: [ss_sold_date_sk#64] +Right keys [1]: [d_date_sk#67] Join condition: None (85) Project [codegen id : 3] -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#73] +Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] +Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67] (86) HashAggregate [codegen id : 3] -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#74, isEmpty#75] -Results [3]: [c_customer_sk#72, sum#76, isEmpty#77] +Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] +Keys [1]: [c_customer_sk#66] +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#68, isEmpty#69] +Results [3]: [c_customer_sk#66, sum#70, isEmpty#71] (87) Exchange -Input [3]: [c_customer_sk#72, sum#76, isEmpty#77] -Arguments: hashpartitioning(c_customer_sk#72, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] +Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, [id=#72] (88) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#72, sum#76, isEmpty#77] -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))#79] -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))#79 AS csales#80] +Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] +Keys [1]: [c_customer_sk#66] +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))#73] +Results [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))#73 AS csales#74] (89) HashAggregate [codegen id : 4] -Input [1]: [csales#80] +Input [1]: [csales#74] Keys: [] -Functions [1]: [partial_max(csales#80)] -Aggregate Attributes [1]: [max#81] -Results [1]: [max#82] +Functions [1]: [partial_max(csales#74)] +Aggregate Attributes [1]: [max#75] +Results [1]: [max#76] (90) Exchange -Input [1]: [max#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] +Input [1]: [max#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (91) HashAggregate [codegen id : 5] -Input [1]: [max#82] +Input [1]: [max#76] Keys: [] -Functions [1]: [max(csales#80)] -Aggregate Attributes [1]: [max(csales#80)#84] -Results [1]: [max(csales#80)#84 AS tpcds_cmax#85] +Functions [1]: [max(csales#74)] +Aggregate Attributes [1]: [max(csales#74)#78] +Results [1]: [max(csales#74)#78 AS tpcds_cmax#79] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 BroadcastExchange (96) +- * Project (95) +- * Filter (94) @@ -531,26 +531,26 @@ BroadcastExchange (96) (92) Scan parquet default.date_dim -Output [2]: [d_date_sk#73, d_year#86] +Output [2]: [d_date_sk#67, d_year#80] 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 (93) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#73, d_year#86] +Input [2]: [d_date_sk#67, d_year#80] (94) Filter [codegen id : 1] -Input [2]: [d_date_sk#73, d_year#86] -Condition : (d_year#86 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#73)) +Input [2]: [d_date_sk#67, d_year#80] +Condition : (d_year#80 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) (95) Project [codegen id : 1] -Output [1]: [d_date_sk#73] -Input [2]: [d_date_sk#73, d_year#86] +Output [1]: [d_date_sk#67] +Input [2]: [d_date_sk#67, d_year#80] (96) BroadcastExchange -Input [1]: [d_date_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#87] +Input [1]: [d_date_sk#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#81] Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 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 115351bfbc776..638f5ec3ded62 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 @@ -489,41 +489,41 @@ Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price Arguments: [ws_item_sk#51 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 12] -Output [2]: [ss_item_sk#57, d_date#58] +Output [2]: [ss_item_sk#8, d_date#12] (80) Sort [codegen id : 32] -Input [2]: [ss_item_sk#57, d_date#58] -Arguments: [ss_item_sk#57 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#8, d_date#12] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 17] -Output [2]: [i_item_sk#59, i_item_desc#60] +Output [2]: [i_item_sk#14, i_item_desc#15] (82) Sort [codegen id : 34] -Input [2]: [i_item_sk#59, i_item_desc#60] -Arguments: [i_item_sk#59 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#14, i_item_desc#15] +Arguments: [i_item_sk#14 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#57] -Right keys [1]: [i_item_sk#59] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#14] Join condition: None (84) Project [codegen id : 35] -Output [3]: [d_date#58, i_item_sk#59, substr(i_item_desc#60, 1, 30) AS _groupingexpression#61] -Input [4]: [ss_item_sk#57, d_date#58, i_item_sk#59, i_item_desc#60] +Output [3]: [d_date#12, i_item_sk#14, substr(i_item_desc#15, 1, 30) AS _groupingexpression#57] +Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#14, i_item_desc#15] (85) HashAggregate [codegen id : 35] -Input [3]: [d_date#58, i_item_sk#59, _groupingexpression#61] -Keys [3]: [_groupingexpression#61, i_item_sk#59, d_date#58] +Input [3]: [d_date#12, i_item_sk#14, _groupingexpression#57] +Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#62] -Results [4]: [_groupingexpression#61, i_item_sk#59, d_date#58, count#63] +Aggregate Attributes [1]: [count#18] +Results [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] (86) HashAggregate [codegen id : 35] -Input [4]: [_groupingexpression#61, i_item_sk#59, d_date#58, count#63] -Keys [3]: [_groupingexpression#61, i_item_sk#59, d_date#58] +Input [4]: [_groupingexpression#57, i_item_sk#14, d_date#12, count#19] +Keys [3]: [_groupingexpression#57, i_item_sk#14, d_date#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#64] -Results [2]: [i_item_sk#59 AS item_sk#21, count(1)#64 AS cnt#22] +Aggregate Attributes [1]: [count(1)#20] +Results [2]: [i_item_sk#14 AS item_sk#21, count(1)#20 AS cnt#22] (87) Filter [codegen id : 35] Input [2]: [item_sk#21, cnt#22] @@ -548,165 +548,165 @@ Input [5]: [ws_item_sk#51, ws_bill_customer_sk#52, ws_quantity#53, ws_list_price (92) Exchange Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] -Arguments: hashpartitioning(ws_bill_customer_sk#52, 5), ENSURE_REQUIREMENTS, [id=#65] +Arguments: hashpartitioning(ws_bill_customer_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] (93) Sort [codegen id : 37] Input [4]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55] Arguments: [ws_bill_customer_sk#52 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 34] -Output [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] (95) Sort [codegen id : 39] -Input [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] -Arguments: [ss_customer_sk#66 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#69] +Output [1]: [c_customer_sk#29] (97) Sort [codegen id : 41] -Input [1]: [c_customer_sk#69] -Arguments: [c_customer_sk#69 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 42] -Left keys [1]: [ss_customer_sk#66] -Right keys [1]: [c_customer_sk#69] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#29] Join condition: None (99) Project [codegen id : 42] -Output [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#69] -Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, c_customer_sk#69] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] (100) HashAggregate [codegen id : 42] -Input [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#69] -Keys [1]: [c_customer_sk#69] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [3]: [c_customer_sk#69, sum#72, isEmpty#73] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (101) HashAggregate [codegen id : 42] -Input [3]: [c_customer_sk#69, sum#72, isEmpty#73] -Keys [1]: [c_customer_sk#69] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))#74] -Results [2]: [c_customer_sk#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))#74 AS ssales#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 AS ssales#36] (102) Filter [codegen id : 42] -Input [2]: [c_customer_sk#69, ssales#36] +Input [2]: [c_customer_sk#29, ssales#36] Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) (103) Project [codegen id : 42] -Output [1]: [c_customer_sk#69] -Input [2]: [c_customer_sk#69, ssales#36] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, ssales#36] (104) Sort [codegen id : 42] -Input [1]: [c_customer_sk#69] -Arguments: [c_customer_sk#69 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [ws_bill_customer_sk#52] -Right keys [1]: [c_customer_sk#69] +Right keys [1]: [c_customer_sk#29] Join condition: None (106) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#75] +Output [1]: [d_date_sk#59] (107) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#55] -Right keys [1]: [d_date_sk#75] +Right keys [1]: [d_date_sk#59] Join condition: None (108) Project [codegen id : 44] Output [3]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54] -Input [5]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55, d_date_sk#75] +Input [5]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, ws_sold_date_sk#55, d_date_sk#59] (109) ReusedExchange [Reuses operator id: 55] -Output [3]: [c_customer_sk#76, c_first_name#77, c_last_name#78] +Output [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] (110) Sort [codegen id : 46] -Input [3]: [c_customer_sk#76, c_first_name#77, c_last_name#78] -Arguments: [c_customer_sk#76 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#60, c_first_name#61, c_last_name#62] +Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 (111) ReusedExchange [Reuses operator id: 34] -Output [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] +Output [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] (112) Sort [codegen id : 48] -Input [3]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68] -Arguments: [ss_customer_sk#66 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26] +Arguments: [ss_customer_sk#24 ASC NULLS FIRST], false, 0 (113) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#69] +Output [1]: [c_customer_sk#29] (114) Sort [codegen id : 50] -Input [1]: [c_customer_sk#69] -Arguments: [c_customer_sk#69 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (115) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#66] -Right keys [1]: [c_customer_sk#69] +Left keys [1]: [ss_customer_sk#24] +Right keys [1]: [c_customer_sk#29] Join condition: None (116) Project [codegen id : 51] -Output [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#69] -Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, c_customer_sk#69] +Output [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] +Input [4]: [ss_customer_sk#24, ss_quantity#25, ss_sales_price#26, c_customer_sk#29] (117) HashAggregate [codegen id : 51] -Input [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#69] -Keys [1]: [c_customer_sk#69] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [3]: [c_customer_sk#69, sum#72, isEmpty#73] +Input [3]: [ss_quantity#25, ss_sales_price#26, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (118) HashAggregate [codegen id : 51] -Input [3]: [c_customer_sk#69, sum#72, isEmpty#73] -Keys [1]: [c_customer_sk#69] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))#74] -Results [2]: [c_customer_sk#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#67 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#68 as decimal(12,2)))), DecimalType(18,2), true))#74 AS ssales#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 AS ssales#36] (119) Filter [codegen id : 51] -Input [2]: [c_customer_sk#69, ssales#36] +Input [2]: [c_customer_sk#29, ssales#36] Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) (120) Project [codegen id : 51] -Output [1]: [c_customer_sk#69] -Input [2]: [c_customer_sk#69, ssales#36] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, ssales#36] (121) Sort [codegen id : 51] -Input [1]: [c_customer_sk#69] -Arguments: [c_customer_sk#69 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (122) SortMergeJoin [codegen id : 52] -Left keys [1]: [c_customer_sk#76] -Right keys [1]: [c_customer_sk#69] +Left keys [1]: [c_customer_sk#60] +Right keys [1]: [c_customer_sk#29] Join condition: None (123) SortMergeJoin [codegen id : 53] Left keys [1]: [ws_bill_customer_sk#52] -Right keys [1]: [c_customer_sk#76] +Right keys [1]: [c_customer_sk#60] Join condition: None (124) Project [codegen id : 53] -Output [4]: [ws_quantity#53, ws_list_price#54, c_first_name#77, c_last_name#78] -Input [6]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, c_customer_sk#76, c_first_name#77, c_last_name#78] +Output [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] +Input [6]: [ws_bill_customer_sk#52, ws_quantity#53, ws_list_price#54, c_customer_sk#60, c_first_name#61, c_last_name#62] (125) HashAggregate [codegen id : 53] -Input [4]: [ws_quantity#53, ws_list_price#54, c_first_name#77, c_last_name#78] -Keys [2]: [c_last_name#78, c_first_name#77] +Input [4]: [ws_quantity#53, ws_list_price#54, c_first_name#61, c_last_name#62] +Keys [2]: [c_last_name#62, c_first_name#61] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#79, isEmpty#80] -Results [4]: [c_last_name#78, c_first_name#77, sum#81, isEmpty#82] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] (126) Exchange -Input [4]: [c_last_name#78, c_first_name#77, sum#81, isEmpty#82] -Arguments: hashpartitioning(c_last_name#78, c_first_name#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +Arguments: hashpartitioning(c_last_name#62, c_first_name#61, 5), ENSURE_REQUIREMENTS, [id=#67] (127) HashAggregate [codegen id : 54] -Input [4]: [c_last_name#78, c_first_name#77, sum#81, isEmpty#82] -Keys [2]: [c_last_name#78, c_first_name#77] +Input [4]: [c_last_name#62, c_first_name#61, sum#65, isEmpty#66] +Keys [2]: [c_last_name#62, c_first_name#61] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#84] -Results [3]: [c_last_name#78, c_first_name#77, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#85] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68] +Results [3]: [c_last_name#62, c_first_name#61, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#53 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#54 as decimal(12,2)))), DecimalType(18,2), true))#68 AS sales#69] (128) Union @@ -725,26 +725,26 @@ BroadcastExchange (134) (130) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#86, d_moy#87] +Output [3]: [d_date_sk#39, d_year#70, d_moy#71] 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 (131) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#86, d_moy#87] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] (132) Filter [codegen id : 1] -Input [3]: [d_date_sk#39, d_year#86, d_moy#87] -Condition : ((((isnotnull(d_year#86) AND isnotnull(d_moy#87)) AND (d_year#86 = 2000)) AND (d_moy#87 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] +Condition : ((((isnotnull(d_year#70) AND isnotnull(d_moy#71)) AND (d_year#70 = 2000)) AND (d_moy#71 = 2)) AND isnotnull(d_date_sk#39)) (133) Project [codegen id : 1] Output [1]: [d_date_sk#39] -Input [3]: [d_date_sk#39, d_year#86, d_moy#87] +Input [3]: [d_date_sk#39, d_year#70, d_moy#71] (134) BroadcastExchange Input [1]: [d_date_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#88] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (139) @@ -755,26 +755,26 @@ BroadcastExchange (139) (135) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_date#12, d_year#89] +Output [3]: [d_date_sk#11, d_date#12, d_year#73] 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 (136) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#89] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] (137) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_date#12, d_year#89] -Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_date#12, d_year#73] +Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (138) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_date#12] -Input [3]: [d_date_sk#11, d_date#12, d_year#89] +Input [3]: [d_date_sk#11, d_date#12, d_year#73] (139) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#74] Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquery#37, [id=#38] * HashAggregate (156) @@ -797,89 +797,89 @@ Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94] +Output [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#94), dynamicpruningexpression(ss_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, ss_sold_date_sk#94] -Condition : isnotnull(ss_customer_sk#91) +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_customer_sk#75) (143) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#96] +Output [1]: [d_date_sk#80] (144) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#94] -Right keys [1]: [d_date_sk#96] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#80] Join condition: None (145) Project [codegen id : 2] -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] +Output [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Input [5]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, ss_sold_date_sk#78, d_date_sk#80] (146) Exchange -Input [3]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93] -Arguments: hashpartitioning(ss_customer_sk#91, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Arguments: hashpartitioning(ss_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (147) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93] -Arguments: [ss_customer_sk#91 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77] +Arguments: [ss_customer_sk#75 ASC NULLS FIRST], false, 0 (148) ReusedExchange [Reuses operator id: 39] -Output [1]: [c_customer_sk#98] +Output [1]: [c_customer_sk#82] (149) Sort [codegen id : 5] -Input [1]: [c_customer_sk#98] -Arguments: [c_customer_sk#98 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#82] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (150) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#91] -Right keys [1]: [c_customer_sk#98] +Left keys [1]: [ss_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (151) Project [codegen id : 6] -Output [3]: [ss_quantity#92, ss_sales_price#93, c_customer_sk#98] -Input [4]: [ss_customer_sk#91, ss_quantity#92, ss_sales_price#93, c_customer_sk#98] +Output [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] +Input [4]: [ss_customer_sk#75, ss_quantity#76, ss_sales_price#77, c_customer_sk#82] (152) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#92, ss_sales_price#93, c_customer_sk#98] -Keys [1]: [c_customer_sk#98] -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#99, isEmpty#100] -Results [3]: [c_customer_sk#98, sum#101, isEmpty#102] +Input [3]: [ss_quantity#76, ss_sales_price#77, c_customer_sk#82] +Keys [1]: [c_customer_sk#82] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [3]: [c_customer_sk#82, sum#85, isEmpty#86] (153) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#98, sum#101, isEmpty#102] -Keys [1]: [c_customer_sk#98] -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))#103] -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))#103 AS csales#104] +Input [3]: [c_customer_sk#82, sum#85, isEmpty#86] +Keys [1]: [c_customer_sk#82] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#77 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] (154) HashAggregate [codegen id : 6] -Input [1]: [csales#104] +Input [1]: [csales#88] Keys: [] -Functions [1]: [partial_max(csales#104)] -Aggregate Attributes [1]: [max#105] -Results [1]: [max#106] +Functions [1]: [partial_max(csales#88)] +Aggregate Attributes [1]: [max#89] +Results [1]: [max#90] (155) Exchange -Input [1]: [max#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [1]: [max#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] (156) HashAggregate [codegen id : 7] -Input [1]: [max#106] +Input [1]: [max#90] Keys: [] -Functions [1]: [max(csales#104)] -Aggregate Attributes [1]: [max(csales#104)#108] -Results [1]: [max(csales#104)#108 AS tpcds_cmax#109] +Functions [1]: [max(csales#88)] +Aggregate Attributes [1]: [max(csales#88)#92] +Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#94 IN dynamicpruning#95 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 BroadcastExchange (161) +- * Project (160) +- * Filter (159) @@ -888,26 +888,26 @@ BroadcastExchange (161) (157) Scan parquet default.date_dim -Output [2]: [d_date_sk#96, d_year#110] +Output [2]: [d_date_sk#80, d_year#94] 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 (158) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#96, d_year#110] +Input [2]: [d_date_sk#80, d_year#94] (159) Filter [codegen id : 1] -Input [2]: [d_date_sk#96, d_year#110] -Condition : (d_year#110 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#96)) +Input [2]: [d_date_sk#80, d_year#94] +Condition : (d_year#94 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#80)) (160) Project [codegen id : 1] -Output [1]: [d_date_sk#96] -Input [2]: [d_date_sk#96, d_year#110] +Output [1]: [d_date_sk#80] +Input [2]: [d_date_sk#80, d_year#94] (161) BroadcastExchange -Input [1]: [d_date_sk#96] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#111] +Input [1]: [d_date_sk#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#95] Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] 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 535d6974834f0..371f34bc14b4b 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 @@ -412,73 +412,73 @@ Input [4]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, ws_sold_da Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 (71) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#58, sum#59, isEmpty#60] +Output [3]: [c_customer_sk#28, sum#32, isEmpty#33] (72) HashAggregate [codegen id : 27] -Input [3]: [c_customer_sk#58, sum#59, isEmpty#60] -Keys [1]: [c_customer_sk#58] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#62 as decimal(12,2)))), DecimalType(18,2), true))#63] -Results [2]: [c_customer_sk#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#62 as decimal(12,2)))), DecimalType(18,2), true))#63 AS ssales#36] +Input [3]: [c_customer_sk#28, sum#32, isEmpty#33] +Keys [1]: [c_customer_sk#28] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35] +Results [2]: [c_customer_sk#28, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#25 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#26 as decimal(12,2)))), DecimalType(18,2), true))#35 AS ssales#36] (73) Filter [codegen id : 27] -Input [2]: [c_customer_sk#58, ssales#36] +Input [2]: [c_customer_sk#28, ssales#36] Condition : (isnotnull(ssales#36) AND (cast(ssales#36 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#37, [id=#38] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 27] -Output [1]: [c_customer_sk#58] -Input [2]: [c_customer_sk#58, ssales#36] +Output [1]: [c_customer_sk#28] +Input [2]: [c_customer_sk#28, ssales#36] (75) Sort [codegen id : 27] -Input [1]: [c_customer_sk#58] -Arguments: [c_customer_sk#58 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (76) SortMergeJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#53] -Right keys [1]: [c_customer_sk#58] +Right keys [1]: [c_customer_sk#28] Join condition: None (77) ReusedExchange [Reuses operator id: 54] -Output [3]: [c_customer_sk#64, c_first_name#65, c_last_name#66] +Output [3]: [c_customer_sk#58, c_first_name#59, c_last_name#60] (78) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_bill_customer_sk#53] -Right keys [1]: [c_customer_sk#64] +Right keys [1]: [c_customer_sk#58] Join condition: None (79) Project [codegen id : 35] -Output [5]: [ws_quantity#54, ws_list_price#55, ws_sold_date_sk#56, c_first_name#65, c_last_name#66] -Input [7]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, ws_sold_date_sk#56, c_customer_sk#64, c_first_name#65, c_last_name#66] +Output [5]: [ws_quantity#54, ws_list_price#55, ws_sold_date_sk#56, c_first_name#59, c_last_name#60] +Input [7]: [ws_bill_customer_sk#53, ws_quantity#54, ws_list_price#55, ws_sold_date_sk#56, c_customer_sk#58, c_first_name#59, c_last_name#60] (80) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#61] (81) BroadcastHashJoin [codegen id : 35] Left keys [1]: [ws_sold_date_sk#56] -Right keys [1]: [d_date_sk#67] +Right keys [1]: [d_date_sk#61] Join condition: None (82) Project [codegen id : 35] -Output [4]: [ws_quantity#54, ws_list_price#55, c_first_name#65, c_last_name#66] -Input [6]: [ws_quantity#54, ws_list_price#55, ws_sold_date_sk#56, c_first_name#65, c_last_name#66, d_date_sk#67] +Output [4]: [ws_quantity#54, ws_list_price#55, c_first_name#59, c_last_name#60] +Input [6]: [ws_quantity#54, ws_list_price#55, ws_sold_date_sk#56, c_first_name#59, c_last_name#60, d_date_sk#61] (83) HashAggregate [codegen id : 35] -Input [4]: [ws_quantity#54, ws_list_price#55, c_first_name#65, c_last_name#66] -Keys [2]: [c_last_name#66, c_first_name#65] +Input [4]: [ws_quantity#54, ws_list_price#55, c_first_name#59, c_last_name#60] +Keys [2]: [c_last_name#60, c_first_name#59] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [4]: [c_last_name#66, c_first_name#65, sum#70, isEmpty#71] +Aggregate Attributes [2]: [sum#62, isEmpty#63] +Results [4]: [c_last_name#60, c_first_name#59, sum#64, isEmpty#65] (84) Exchange -Input [4]: [c_last_name#66, c_first_name#65, sum#70, isEmpty#71] -Arguments: hashpartitioning(c_last_name#66, c_first_name#65, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [4]: [c_last_name#60, c_first_name#59, sum#64, isEmpty#65] +Arguments: hashpartitioning(c_last_name#60, c_first_name#59, 5), ENSURE_REQUIREMENTS, [id=#66] (85) HashAggregate [codegen id : 36] -Input [4]: [c_last_name#66, c_first_name#65, sum#70, isEmpty#71] -Keys [2]: [c_last_name#66, c_first_name#65] +Input [4]: [c_last_name#60, c_first_name#59, sum#64, isEmpty#65] +Keys [2]: [c_last_name#60, c_first_name#59] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#73] -Results [3]: [c_last_name#66, c_first_name#65, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#73 AS sales#74] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#67] +Results [3]: [c_last_name#60, c_first_name#59, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#54 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#55 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#68] (86) Union @@ -497,26 +497,26 @@ BroadcastExchange (92) (88) Scan parquet default.date_dim -Output [3]: [d_date_sk#44, d_year#75, d_moy#76] +Output [3]: [d_date_sk#44, d_year#69, d_moy#70] 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 (89) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#44, d_year#75, d_moy#76] +Input [3]: [d_date_sk#44, d_year#69, d_moy#70] (90) Filter [codegen id : 1] -Input [3]: [d_date_sk#44, d_year#75, d_moy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_moy#76)) AND (d_year#75 = 2000)) AND (d_moy#76 = 2)) AND isnotnull(d_date_sk#44)) +Input [3]: [d_date_sk#44, d_year#69, d_moy#70] +Condition : ((((isnotnull(d_year#69) AND isnotnull(d_moy#70)) AND (d_year#69 = 2000)) AND (d_moy#70 = 2)) AND isnotnull(d_date_sk#44)) (91) Project [codegen id : 1] Output [1]: [d_date_sk#44] -Input [3]: [d_date_sk#44, d_year#75, d_moy#76] +Input [3]: [d_date_sk#44, d_year#69, d_moy#70] (92) BroadcastExchange Input [1]: [d_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (97) @@ -527,26 +527,26 @@ BroadcastExchange (97) (93) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#78] +Output [3]: [d_date_sk#10, d_date#11, d_year#72] 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 (94) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#78] +Input [3]: [d_date_sk#10, d_date#11, d_year#72] (95) Filter [codegen id : 1] -Input [3]: [d_date_sk#10, d_date#11, d_year#78] -Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#10, d_date#11, d_year#72] +Condition : (d_year#72 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (96) Project [codegen id : 1] Output [2]: [d_date_sk#10, d_date#11] -Input [3]: [d_date_sk#10, d_date#11, d_year#78] +Input [3]: [d_date_sk#10, d_date#11, d_year#72] (97) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#73] Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#37, [id=#38] * HashAggregate (112) @@ -567,81 +567,81 @@ Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquer (98) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83] +Output [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (99) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77] (100) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#80, ss_quantity#81, ss_sales_price#82, ss_sold_date_sk#83] -Condition : isnotnull(ss_customer_sk#80) +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77] +Condition : isnotnull(ss_customer_sk#74) (101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#85] +Output [1]: [c_customer_sk#79] (102) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#80] -Right keys [1]: [c_customer_sk#85] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#79] Join condition: None (103) Project [codegen id : 3] -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] +Output [4]: [ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77, c_customer_sk#79] +Input [5]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77, c_customer_sk#79] (104) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#86] +Output [1]: [d_date_sk#80] (105) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#86] +Left keys [1]: [ss_sold_date_sk#77] +Right keys [1]: [d_date_sk#80] Join condition: None (106) Project [codegen id : 3] -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#86] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#79] +Input [5]: [ss_quantity#75, ss_sales_price#76, ss_sold_date_sk#77, c_customer_sk#79, d_date_sk#80] (107) HashAggregate [codegen id : 3] -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#87, isEmpty#88] -Results [3]: [c_customer_sk#85, sum#89, isEmpty#90] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#79] +Keys [1]: [c_customer_sk#79] +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#81, isEmpty#82] +Results [3]: [c_customer_sk#79, sum#83, isEmpty#84] (108) Exchange -Input [3]: [c_customer_sk#85, sum#89, isEmpty#90] -Arguments: hashpartitioning(c_customer_sk#85, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [3]: [c_customer_sk#79, sum#83, isEmpty#84] +Arguments: hashpartitioning(c_customer_sk#79, 5), ENSURE_REQUIREMENTS, [id=#85] (109) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#85, sum#89, isEmpty#90] -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))#92] -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))#92 AS csales#93] +Input [3]: [c_customer_sk#79, sum#83, isEmpty#84] +Keys [1]: [c_customer_sk#79] +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))#86] +Results [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))#86 AS csales#87] (110) HashAggregate [codegen id : 4] -Input [1]: [csales#93] +Input [1]: [csales#87] Keys: [] -Functions [1]: [partial_max(csales#93)] -Aggregate Attributes [1]: [max#94] -Results [1]: [max#95] +Functions [1]: [partial_max(csales#87)] +Aggregate Attributes [1]: [max#88] +Results [1]: [max#89] (111) Exchange -Input [1]: [max#95] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] +Input [1]: [max#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (112) HashAggregate [codegen id : 5] -Input [1]: [max#95] +Input [1]: [max#89] Keys: [] -Functions [1]: [max(csales#93)] -Aggregate Attributes [1]: [max(csales#93)#97] -Results [1]: [max(csales#93)#97 AS tpcds_cmax#98] +Functions [1]: [max(csales#87)] +Aggregate Attributes [1]: [max(csales#87)#91] +Results [1]: [max(csales#87)#91 AS tpcds_cmax#92] -Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#78 BroadcastExchange (117) +- * Project (116) +- * Filter (115) @@ -650,26 +650,26 @@ BroadcastExchange (117) (113) Scan parquet default.date_dim -Output [2]: [d_date_sk#86, d_year#99] +Output [2]: [d_date_sk#80, d_year#93] 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 (114) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#86, d_year#99] +Input [2]: [d_date_sk#80, d_year#93] (115) Filter [codegen id : 1] -Input [2]: [d_date_sk#86, d_year#99] -Condition : (d_year#99 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#86)) +Input [2]: [d_date_sk#80, d_year#93] +Condition : (d_year#93 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#80)) (116) Project [codegen id : 1] -Output [1]: [d_date_sk#86] -Input [2]: [d_date_sk#86, d_year#99] +Output [1]: [d_date_sk#80] +Input [2]: [d_date_sk#80, d_year#93] (117) BroadcastExchange -Input [1]: [d_date_sk#86] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#100] +Input [1]: [d_date_sk#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] 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 b57eca2578e00..2ecb115faf87d 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 @@ -328,214 +328,214 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (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] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] 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 : 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] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (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)) +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) (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] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (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] +Output [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] 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 (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] +Input [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] (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)) +Input [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] +Condition : (((isnotnull(s_market_id#27) AND (s_market_id#27 = 8)) AND isnotnull(s_store_sk#25)) AND isnotnull(s_zip#29)) (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] +Output [4]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29] +Input [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] (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] +Input [4]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (59) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#55] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#25] Join condition: None (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] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#25, s_store_name#26, s_state#28, s_zip#29] (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] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#50] (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 +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (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] +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (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] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (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) +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : isnotnull(i_item_sk#7) (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] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] (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 +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#62] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] Join condition: None (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] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (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] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#52] (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 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 16] -Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (73) 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 +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#70] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] Join condition: None (75) 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] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (76) 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=#74] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] (77) 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 +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 26] -Output [2]: [sr_item_sk#75, sr_ticket_number#76] +Output [2]: [sr_item_sk#21, sr_ticket_number#22] (79) Sort [codegen id : 13] -Input [2]: [sr_item_sk#75, sr_ticket_number#76] -Arguments: [sr_ticket_number#76 ASC NULLS FIRST, sr_item_sk#75 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#21, sr_ticket_number#22] +Arguments: [sr_ticket_number#22 ASC NULLS FIRST, sr_item_sk#21 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#76, sr_item_sk#75] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#22, sr_item_sk#21] Join condition: None (81) 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#75, sr_ticket_number#76] +Output [12]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [16]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, sr_item_sk#21, sr_ticket_number#22] (82) 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=#77] +Input [12]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_birth_country#18, s_zip#29, 5), ENSURE_REQUIREMENTS, [id=#54] (83) 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 +Input [12]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#29 ASC NULLS FIRST], false, 0 (84) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#31, ca_zip#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (85) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] (86) Filter [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] +Condition : (isnotnull(ca_country#33) AND isnotnull(ca_zip#32)) (87) Exchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: hashpartitioning(upper(ca_country#80), ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] +Arguments: hashpartitioning(upper(ca_country#33), ca_zip#32, 5), ENSURE_REQUIREMENTS, [id=#55] (88) Sort [codegen id : 17] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: [upper(ca_country#80) ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] +Arguments: [upper(ca_country#33) ASC NULLS FIRST, ca_zip#32 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#73, s_zip#59] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +Left keys [2]: [c_birth_country#18, s_zip#29] +Right keys [2]: [upper(ca_country#33), ca_zip#32] Join condition: None (90) 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#78] -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#78, ca_zip#79, ca_country#80] +Output [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] +Input [15]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, ca_state#31, ca_zip#32, ca_country#33] (91) 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#78] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, 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#82] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] +Input [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#56] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#57] (92) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#57] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#58] (93) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, 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))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#39] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#57] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] (94) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#59, count#60] +Results [2]: [sum#61, count#62] (95) Exchange -Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +Input [2]: [sum#61, count#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (96) HashAggregate [codegen id : 20] -Input [2]: [sum#88, count#89] +Input [2]: [sum#61, count#62] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#39)#64] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] 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 c8d73ab27c82b..0ad7d96f8f777 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 @@ -293,125 +293,125 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (47) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] (48) 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 +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (49) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#53, sr_ticket_number#54] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] (50) Sort [codegen id : 4] -Input [2]: [sr_item_sk#53, sr_ticket_number#54] -Arguments: [sr_ticket_number#54 ASC NULLS FIRST, sr_item_sk#53 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: [sr_ticket_number#9 ASC NULLS FIRST, sr_item_sk#8 ASC NULLS FIRST], false, 0 (51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] -Right keys [2]: [sr_ticket_number#54, sr_item_sk#53] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (52) 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#53, sr_ticket_number#54] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#8, sr_ticket_number#9] (53) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] (54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#50] -Right keys [1]: [s_store_sk#55] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (55) Project [codegen id : 9] -Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58] -Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] (56) Scan parquet default.item -Output [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (58) Filter [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Condition : isnotnull(i_item_sk#59) +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Condition : isnotnull(i_item_sk#18) (59) BroadcastExchange -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] (60) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#48] -Right keys [1]: [i_item_sk#59] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (61) Project [codegen id : 9] -Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (62) ReusedExchange [Reuses operator id: 31] -Output [4]: [c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (63) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#49] -Right keys [1]: [c_customer_sk#66] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (64) Project [codegen id : 9] -Output [12]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69] -Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +Output [12]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (65) ReusedExchange [Reuses operator id: 37] -Output [3]: [ca_state#70, ca_zip#71, ca_country#72] +Output [3]: [ca_state#30, ca_zip#31, ca_country#32] (66) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#69, s_zip#58] -Right keys [2]: [upper(ca_country#72), ca_zip#71] +Left keys [2]: [c_birth_country#28, s_zip#16] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None (67) Project [codegen id : 9] -Output [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Input [15]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69, ca_state#70, ca_zip#71, ca_country#72] +Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] +Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, ca_country#32] (68) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum#73] -Results [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] +Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] +Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#49] +Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#50] (69) Exchange -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Arguments: hashpartitioning(c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#50] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#51] (70) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#76] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#76,17,2) AS netpaid#38] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#50] +Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#37] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#37,17,2) AS netpaid#38] (71) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#77, count#78] -Results [2]: [sum#79, count#80] +Aggregate Attributes [2]: [sum#52, count#53] +Results [2]: [sum#54, count#55] (72) Exchange -Input [2]: [sum#79, count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [2]: [sum#54, count#55] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#56] (73) HashAggregate [codegen id : 11] -Input [2]: [sum#79, count#80] +Input [2]: [sum#54, count#55] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#82] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#82)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#83] +Aggregate Attributes [1]: [avg(netpaid#38)#57] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#57)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#58] 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 1814c7de8204d..9e4e27f2c6726 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 @@ -328,214 +328,214 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (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] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] 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 : 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] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (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)) +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) (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] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (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] +Output [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] 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 (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] +Input [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] (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)) +Input [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] +Condition : (((isnotnull(s_market_id#27) AND (s_market_id#27 = 8)) AND isnotnull(s_store_sk#25)) AND isnotnull(s_zip#29)) (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] +Output [4]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29] +Input [5]: [s_store_sk#25, s_store_name#26, s_market_id#27, s_state#28, s_zip#29] (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] +Input [4]: [s_store_sk#25, s_store_name#26, s_state#28, s_zip#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (59) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#55] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#25] Join condition: None (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] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#25, s_store_name#26, s_state#28, s_zip#29] (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] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#50] (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 +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (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] +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (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] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (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) +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : isnotnull(i_item_sk#7) (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] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] (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 +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#62] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] Join condition: None (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] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (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] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#52] (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 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 16] -Output [4]: [c_customer_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (73) 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 +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#70] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] Join condition: None (75) 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] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (76) 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=#74] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] (77) 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 +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 26] -Output [2]: [sr_item_sk#75, sr_ticket_number#76] +Output [2]: [sr_item_sk#21, sr_ticket_number#22] (79) Sort [codegen id : 13] -Input [2]: [sr_item_sk#75, sr_ticket_number#76] -Arguments: [sr_ticket_number#76 ASC NULLS FIRST, sr_item_sk#75 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#21, sr_ticket_number#22] +Arguments: [sr_ticket_number#22 ASC NULLS FIRST, sr_item_sk#21 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 14] -Left keys [2]: [ss_ticket_number#52, ss_item_sk#49] -Right keys [2]: [sr_ticket_number#76, sr_item_sk#75] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#22, sr_item_sk#21] Join condition: None (81) 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#75, sr_ticket_number#76] +Output [12]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [16]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, sr_item_sk#21, sr_ticket_number#22] (82) 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=#77] +Input [12]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_birth_country#18, s_zip#29, 5), ENSURE_REQUIREMENTS, [id=#54] (83) 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 +Input [12]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#29 ASC NULLS FIRST], false, 0 (84) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#31, ca_zip#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (85) ColumnarToRow [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] (86) Filter [codegen id : 16] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] +Condition : (isnotnull(ca_country#33) AND isnotnull(ca_zip#32)) (87) Exchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: hashpartitioning(upper(ca_country#80), ca_zip#79, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] +Arguments: hashpartitioning(upper(ca_country#33), ca_zip#32, 5), ENSURE_REQUIREMENTS, [id=#55] (88) Sort [codegen id : 17] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: [upper(ca_country#80) ASC NULLS FIRST, ca_zip#79 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#31, ca_zip#32, ca_country#33] +Arguments: [upper(ca_country#33) ASC NULLS FIRST, ca_zip#32 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 18] -Left keys [2]: [c_birth_country#73, s_zip#59] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +Left keys [2]: [c_birth_country#18, s_zip#29] +Right keys [2]: [upper(ca_country#33), ca_zip#32] Join condition: None (90) 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#78] -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#78, ca_zip#79, ca_country#80] +Output [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] +Input [15]: [ss_net_paid#5, s_store_name#26, s_state#28, s_zip#29, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, ca_state#31, ca_zip#32, ca_country#33] (91) 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#78] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, 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#82] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] +Input [11]: [ss_net_paid#5, s_store_name#26, s_state#28, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#31] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#56] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#57] (92) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#57] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#58] (93) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, s_state#58, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#83] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#56, ca_state#78, 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))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#39] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#57] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#26, ca_state#31, s_state#28, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] (94) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#59, count#60] +Results [2]: [sum#61, count#62] (95) Exchange -Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] +Input [2]: [sum#61, count#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (96) HashAggregate [codegen id : 20] -Input [2]: [sum#88, count#89] +Input [2]: [sum#61, count#62] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#39)#64] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] 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 443ca4d35b780..78371d380114e 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 @@ -293,125 +293,125 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (47) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_ticket_number#51, ss_net_paid#52] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] (48) 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 +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (49) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#53, sr_ticket_number#54] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] (50) Sort [codegen id : 4] -Input [2]: [sr_item_sk#53, sr_ticket_number#54] -Arguments: [sr_ticket_number#54 ASC NULLS FIRST, sr_item_sk#53 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: [sr_ticket_number#9 ASC NULLS FIRST, sr_item_sk#8 ASC NULLS FIRST], false, 0 (51) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#51, ss_item_sk#48] -Right keys [2]: [sr_ticket_number#54, sr_item_sk#53] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (52) 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#53, sr_ticket_number#54] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#8, sr_ticket_number#9] (53) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] (54) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#50] -Right keys [1]: [s_store_sk#55] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (55) Project [codegen id : 9] -Output [6]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58] -Input [8]: [ss_item_sk#48, ss_customer_sk#49, ss_store_sk#50, ss_net_paid#52, s_store_sk#55, s_store_name#56, s_state#57, s_zip#58] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] (56) Scan parquet default.item -Output [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (58) Filter [codegen id : 6] -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Condition : isnotnull(i_item_sk#59) +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Condition : isnotnull(i_item_sk#18) (59) BroadcastExchange -Input [6]: [i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] (60) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#48] -Right keys [1]: [i_item_sk#59] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (61) Project [codegen id : 9] -Output [10]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] -Input [12]: [ss_item_sk#48, ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_item_sk#59, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (62) ReusedExchange [Reuses operator id: 31] -Output [4]: [c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (63) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#49] -Right keys [1]: [c_customer_sk#66] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (64) Project [codegen id : 9] -Output [12]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69] -Input [14]: [ss_customer_sk#49, ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_customer_sk#66, c_first_name#67, c_last_name#68, c_birth_country#69] +Output [12]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (65) ReusedExchange [Reuses operator id: 37] -Output [3]: [ca_state#70, ca_zip#71, ca_country#72] +Output [3]: [ca_state#30, ca_zip#31, ca_country#32] (66) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#69, s_zip#58] -Right keys [2]: [upper(ca_country#72), ca_zip#71] +Left keys [2]: [c_birth_country#28, s_zip#16] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None (67) Project [codegen id : 9] -Output [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Input [15]: [ss_net_paid#52, s_store_name#56, s_state#57, s_zip#58, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, c_birth_country#69, ca_state#70, ca_zip#71, ca_country#72] +Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] +Input [15]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, c_birth_country#28, ca_state#30, ca_zip#31, ca_country#32] (68) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#52, s_store_name#56, s_state#57, i_current_price#60, i_size#61, i_color#62, i_units#63, i_manager_id#64, c_first_name#67, c_last_name#68, ca_state#70] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum#73] -Results [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] +Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#26, c_last_name#27, ca_state#30] +Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#49] +Results [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#50] (69) Exchange -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Arguments: hashpartitioning(c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#50] +Arguments: hashpartitioning(c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#51] (70) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61, sum#74] -Keys [10]: [c_last_name#68, c_first_name#67, s_store_name#56, ca_state#70, s_state#57, i_color#62, i_current_price#60, i_manager_id#64, i_units#63, i_size#61] -Functions [1]: [sum(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#52))#76] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#52))#76,17,2) AS netpaid#38] +Input [11]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#50] +Keys [10]: [c_last_name#27, c_first_name#26, s_store_name#13, ca_state#30, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#37] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#37,17,2) AS netpaid#38] (71) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#77, count#78] -Results [2]: [sum#79, count#80] +Aggregate Attributes [2]: [sum#52, count#53] +Results [2]: [sum#54, count#55] (72) Exchange -Input [2]: [sum#79, count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [2]: [sum#54, count#55] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#56] (73) HashAggregate [codegen id : 11] -Input [2]: [sum#79, count#80] +Input [2]: [sum#54, count#55] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#82] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#82)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#83] +Aggregate Attributes [1]: [avg(netpaid#38)#57] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#57)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#58] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt index 7db465072e4c3..35b9877c4fd09 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt @@ -203,108 +203,108 @@ Output [14]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_ Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_customer_sk#33, ctr_state#34, ctr_total_return#35] (33) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] +Output [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#39), dynamicpruningexpression(wr_returned_date_sk#39 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(wr_returned_date_sk#22), dynamicpruningexpression(wr_returned_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 10] -Input [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] +Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] (35) Filter [codegen id : 10] -Input [4]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39] -Condition : isnotnull(wr_returning_addr_sk#37) +Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] +Condition : isnotnull(wr_returning_addr_sk#20) (36) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#24] (37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_returned_date_sk#39] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [wr_returned_date_sk#22] +Right keys [1]: [d_date_sk#24] Join condition: None (38) Project [codegen id : 10] -Output [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] -Input [5]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, wr_returned_date_sk#39, d_date_sk#40] +Output [3]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21] +Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22, d_date_sk#24] (39) Exchange -Input [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] -Arguments: hashpartitioning(wr_returning_addr_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [3]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21] +Arguments: hashpartitioning(wr_returning_addr_sk#20, 5), ENSURE_REQUIREMENTS, [id=#36] (40) Sort [codegen id : 11] -Input [3]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38] -Arguments: [wr_returning_addr_sk#37 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21] +Arguments: [wr_returning_addr_sk#20 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#42, ca_state#43] +Output [2]: [ca_address_sk#26, ca_state#27] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#42, ca_state#43] -Arguments: [ca_address_sk#42 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] -Left keys [1]: [wr_returning_addr_sk#37] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [wr_returning_addr_sk#20] +Right keys [1]: [ca_address_sk#26] Join condition: None (44) Project [codegen id : 14] -Output [3]: [wr_returning_customer_sk#36, wr_return_amt#38, ca_state#43] -Input [5]: [wr_returning_customer_sk#36, wr_returning_addr_sk#37, wr_return_amt#38, ca_address_sk#42, ca_state#43] +Output [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#27] +Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, ca_address_sk#26, ca_state#27] (45) HashAggregate [codegen id : 14] -Input [3]: [wr_returning_customer_sk#36, wr_return_amt#38, ca_state#43] -Keys [2]: [wr_returning_customer_sk#36, ca_state#43] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#38))] -Aggregate Attributes [1]: [sum#44] -Results [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] +Input [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#27] +Keys [2]: [wr_returning_customer_sk#19, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum#37] +Results [3]: [wr_returning_customer_sk#19, ca_state#27, sum#38] (46) Exchange -Input [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] -Arguments: hashpartitioning(wr_returning_customer_sk#36, ca_state#43, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [wr_returning_customer_sk#19, ca_state#27, sum#38] +Arguments: hashpartitioning(wr_returning_customer_sk#19, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#39] (47) HashAggregate [codegen id : 15] -Input [3]: [wr_returning_customer_sk#36, ca_state#43, sum#45] -Keys [2]: [wr_returning_customer_sk#36, ca_state#43] -Functions [1]: [sum(UnscaledValue(wr_return_amt#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#38))#47] -Results [2]: [ca_state#43 AS ctr_state#34, MakeDecimal(sum(UnscaledValue(wr_return_amt#38))#47,17,2) AS ctr_total_return#35] +Input [3]: [wr_returning_customer_sk#19, ca_state#27, sum#38] +Keys [2]: [wr_returning_customer_sk#19, ca_state#27] +Functions [1]: [sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#21))#32] +Results [2]: [ca_state#27 AS ctr_state#34, MakeDecimal(sum(UnscaledValue(wr_return_amt#21))#32,17,2) AS ctr_total_return#35] (48) HashAggregate [codegen id : 15] Input [2]: [ctr_state#34, ctr_total_return#35] Keys [1]: [ctr_state#34] Functions [1]: [partial_avg(ctr_total_return#35)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [3]: [ctr_state#34, sum#50, count#51] +Aggregate Attributes [2]: [sum#40, count#41] +Results [3]: [ctr_state#34, sum#42, count#43] (49) Exchange -Input [3]: [ctr_state#34, sum#50, count#51] -Arguments: hashpartitioning(ctr_state#34, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [3]: [ctr_state#34, sum#42, count#43] +Arguments: hashpartitioning(ctr_state#34, 5), ENSURE_REQUIREMENTS, [id=#44] (50) HashAggregate [codegen id : 16] -Input [3]: [ctr_state#34, sum#50, count#51] +Input [3]: [ctr_state#34, sum#42, count#43] Keys [1]: [ctr_state#34] Functions [1]: [avg(ctr_total_return#35)] -Aggregate Attributes [1]: [avg(ctr_total_return#35)#53] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#35)#53) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#54, ctr_state#34 AS ctr_state#34#55] +Aggregate Attributes [1]: [avg(ctr_total_return#35)#45] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#35)#45) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#46, ctr_state#34 AS ctr_state#34#47] (51) Filter [codegen id : 16] -Input [2]: [(avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#54) +Input [2]: [(avg(ctr_total_return) * 1.2)#46, ctr_state#34#47] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#46) (52) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#56] +Input [2]: [(avg(ctr_total_return) * 1.2)#46, ctr_state#34#47] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#48] (53) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#34] -Right keys [1]: [ctr_state#34#55] -Join condition: (cast(ctr_total_return#35 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#54) +Right keys [1]: [ctr_state#34#47] +Join condition: (cast(ctr_total_return#35 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#46) (54) Project [codegen id : 17] Output [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] -Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, (avg(ctr_total_return) * 1.2)#54, ctr_state#34#55] +Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_state#34, ctr_total_return#35, (avg(ctr_total_return) * 1.2)#46, ctr_state#34#47] (55) TakeOrderedAndProject Input [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] @@ -321,27 +321,27 @@ BroadcastExchange (60) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#57] +Output [2]: [d_date_sk#24, d_year#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#57] +Input [2]: [d_date_sk#24, d_year#49] (58) Filter [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#57] -Condition : ((isnotnull(d_year#57) AND (d_year#57 = 2002)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#24, d_year#49] +Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_sk#24)) (59) Project [codegen id : 1] Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#57] +Input [2]: [d_date_sk#24, d_year#49] (60) BroadcastExchange Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] -Subquery:2 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#39 IN dynamicpruning#23 +Subquery:2 Hosting operator id = 33 Hosting Expression = wr_returned_date_sk#22 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 066b0fa66bf33..fdf276c01e19a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -127,158 +127,158 @@ Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] Condition : isnotnull(ctr_total_return#16) (17) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] +Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#20), dynamicpruningexpression(wr_returned_date_sk#20 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] (19) Filter [codegen id : 6] -Input [4]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20] -Condition : isnotnull(wr_returning_addr_sk#18) +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Condition : isnotnull(wr_returning_addr_sk#2) (20) ReusedExchange [Reuses operator id: 54] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#6] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [wr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (22) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19] -Input [5]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, wr_returned_date_sk#20, d_date_sk#21] +Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] (23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#7, ca_state#8] (24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#18] -Right keys [1]: [ca_address_sk#22] +Left keys [1]: [wr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#7] Join condition: None (25) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#17, wr_return_amt#19, ca_state#23] -Input [5]: [wr_returning_customer_sk#17, wr_returning_addr_sk#18, wr_return_amt#19, ca_address_sk#22, ca_state#23] +Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] (26) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#17, wr_return_amt#19, ca_state#23] -Keys [2]: [wr_returning_customer_sk#17, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#19))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] +Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum#17] +Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#18] (27) Exchange -Input [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] -Arguments: hashpartitioning(wr_returning_customer_sk#17, ca_state#23, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#18] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [id=#19] (28) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#17, ca_state#23, sum#25] -Keys [2]: [wr_returning_customer_sk#17, ca_state#23] -Functions [1]: [sum(UnscaledValue(wr_return_amt#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#19))#27] -Results [2]: [ca_state#23 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(wr_return_amt#19))#27,17,2) AS ctr_total_return#16] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#18] +Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#13] +Results [2]: [ca_state#8 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#13,17,2) AS ctr_total_return#16] (29) HashAggregate [codegen id : 7] Input [2]: [ctr_state#15, ctr_total_return#16] Keys [1]: [ctr_state#15] Functions [1]: [partial_avg(ctr_total_return#16)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ctr_state#15, sum#30, count#31] +Aggregate Attributes [2]: [sum#20, count#21] +Results [3]: [ctr_state#15, sum#22, count#23] (30) Exchange -Input [3]: [ctr_state#15, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [3]: [ctr_state#15, sum#22, count#23] +Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#15, sum#30, count#31] +Input [3]: [ctr_state#15, sum#22, count#23] Keys [1]: [ctr_state#15] Functions [1]: [avg(ctr_total_return#16)] -Aggregate Attributes [1]: [avg(ctr_total_return#16)#33] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#33) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#34, ctr_state#15 AS ctr_state#15#35] +Aggregate Attributes [1]: [avg(ctr_total_return#16)#25] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_state#15 AS ctr_state#15#27] (32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#34) +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_state#15#27] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) (33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#36] +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_state#15#27] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#15] -Right keys [1]: [ctr_state#15#35] -Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#34) +Right keys [1]: [ctr_state#15#27] +Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) (35) Project [codegen id : 11] Output [2]: [ctr_customer_sk#14, ctr_total_return#16] -Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#26, ctr_state#15#27] (36) Scan parquet default.customer -Output [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +Output [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] (38) Filter [codegen id : 9] -Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#39)) +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) (39) BroadcastExchange -Input [14]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#14] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#29] Join condition: None (41) Project [codegen id : 11] -Output [14]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] -Input [16]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50] +Output [14]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Input [16]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] (42) Scan parquet default.customer_address -Output [2]: [ca_address_sk#52, ca_state#53] +Output [2]: [ca_address_sk#44, ca_state#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#52, ca_state#53] +Input [2]: [ca_address_sk#44, ca_state#45] (44) Filter [codegen id : 10] -Input [2]: [ca_address_sk#52, ca_state#53] -Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#52)) +Input [2]: [ca_address_sk#44, ca_state#45] +Condition : ((isnotnull(ca_state#45) AND (ca_state#45 = GA)) AND isnotnull(ca_address_sk#44)) (45) Project [codegen id : 10] -Output [1]: [ca_address_sk#52] -Input [2]: [ca_address_sk#52, ca_state#53] +Output [1]: [ca_address_sk#44] +Input [2]: [ca_address_sk#44, ca_state#45] (46) BroadcastExchange -Input [1]: [ca_address_sk#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [ca_address_sk#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#39] -Right keys [1]: [ca_address_sk#52] +Left keys [1]: [c_current_addr_sk#31] +Right keys [1]: [ca_address_sk#44] Join condition: None (48) Project [codegen id : 11] -Output [13]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] -Input [15]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ca_address_sk#52] +Output [13]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16] +Input [15]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ca_address_sk#44] (49) TakeOrderedAndProject -Input [13]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, c_preferred_cust_flag#43 ASC NULLS FIRST, c_birth_day#44 ASC NULLS FIRST, c_birth_month#45 ASC NULLS FIRST, c_birth_year#46 ASC NULLS FIRST, c_birth_country#47 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#50 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#44, c_birth_month#45, c_birth_year#46, c_birth_country#47, c_login#48, c_email_address#49, c_last_review_date#50, ctr_total_return#16] +Input [13]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16] +Arguments: 100, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, c_preferred_cust_flag#35 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16] ===== Subqueries ===== @@ -291,27 +291,27 @@ BroadcastExchange (54) (50) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#55] +Output [2]: [d_date_sk#6, d_year#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#55] +Input [2]: [d_date_sk#6, d_year#47] (52) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#55] -Condition : ((isnotnull(d_year#55) AND (d_year#55 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_year#47] +Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#6)) (53) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#55] +Input [2]: [d_date_sk#6, d_year#47] (54) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] -Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt index a9569e31b039f..2e78631dfa336 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt @@ -261,12 +261,12 @@ Arguments: hashpartitioning(ca_county#26, d_qoy#23, d_year#22, 5), ENSURE_REQUIR Input [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] Keys [3]: [ca_county#26, d_qoy#23, d_year#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#18))#30] -Results [2]: [ca_county#26, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#18))#30,17,2) AS store_sales#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#18))#15] +Results [2]: [ca_county#26, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#18))#15,17,2) AS store_sales#30] (34) BroadcastExchange -Input [2]: [ca_county#26, store_sales#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] +Input [2]: [ca_county#26, store_sales#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] (35) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#10] @@ -274,344 +274,344 @@ Right keys [1]: [ca_county#26] Join condition: None (36) Project [codegen id : 42] -Output [3]: [ca_county#10, store_sales#16, store_sales#31] -Input [4]: [ca_county#10, store_sales#16, ca_county#26, store_sales#31] +Output [3]: [ca_county#10, store_sales#16, store_sales#30] +Input [4]: [ca_county#10, store_sales#16, ca_county#26, store_sales#30] (37) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Output [3]: [ss_addr_sk#32, ss_ext_sales_price#33, ss_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ss_sold_date_sk#34), dynamicpruningexpression(ss_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 15] -Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Input [3]: [ss_addr_sk#32, ss_ext_sales_price#33, ss_sold_date_sk#34] (39) Filter [codegen id : 15] -Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_addr_sk#33) +Input [3]: [ss_addr_sk#32, ss_ext_sales_price#33, ss_sold_date_sk#34] +Condition : isnotnull(ss_addr_sk#32) (40) ReusedExchange [Reuses operator id: 122] -Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Output [3]: [d_date_sk#36, d_year#37, d_qoy#38] (41) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ss_sold_date_sk#34] +Right keys [1]: [d_date_sk#36] Join condition: None (42) Project [codegen id : 15] -Output [4]: [ss_addr_sk#33, ss_ext_sales_price#34, d_year#38, d_qoy#39] -Input [6]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38, d_qoy#39] +Output [4]: [ss_addr_sk#32, ss_ext_sales_price#33, d_year#37, d_qoy#38] +Input [6]: [ss_addr_sk#32, ss_ext_sales_price#33, ss_sold_date_sk#34, d_date_sk#36, d_year#37, d_qoy#38] (43) Exchange -Input [4]: [ss_addr_sk#33, ss_ext_sales_price#34, d_year#38, d_qoy#39] -Arguments: hashpartitioning(ss_addr_sk#33, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [4]: [ss_addr_sk#32, ss_ext_sales_price#33, d_year#37, d_qoy#38] +Arguments: hashpartitioning(ss_addr_sk#32, 5), ENSURE_REQUIREMENTS, [id=#39] (44) Sort [codegen id : 16] -Input [4]: [ss_addr_sk#33, ss_ext_sales_price#34, d_year#38, d_qoy#39] -Arguments: [ss_addr_sk#33 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#32, ss_ext_sales_price#33, d_year#37, d_qoy#38] +Arguments: [ss_addr_sk#32 ASC NULLS FIRST], false, 0 (45) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#41, ca_county#42] +Output [2]: [ca_address_sk#40, ca_county#41] (46) Sort [codegen id : 18] -Input [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ca_address_sk#41 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#40, ca_county#41] +Arguments: [ca_address_sk#40 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 19] -Left keys [1]: [ss_addr_sk#33] -Right keys [1]: [ca_address_sk#41] +Left keys [1]: [ss_addr_sk#32] +Right keys [1]: [ca_address_sk#40] Join condition: None (48) Project [codegen id : 19] -Output [4]: [ss_ext_sales_price#34, d_year#38, d_qoy#39, ca_county#42] -Input [6]: [ss_addr_sk#33, ss_ext_sales_price#34, d_year#38, d_qoy#39, ca_address_sk#41, ca_county#42] +Output [4]: [ss_ext_sales_price#33, d_year#37, d_qoy#38, ca_county#41] +Input [6]: [ss_addr_sk#32, ss_ext_sales_price#33, d_year#37, d_qoy#38, ca_address_sk#40, ca_county#41] (49) HashAggregate [codegen id : 19] -Input [4]: [ss_ext_sales_price#34, d_year#38, d_qoy#39, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#39, d_year#38] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#43] -Results [4]: [ca_county#42, d_qoy#39, d_year#38, sum#44] +Input [4]: [ss_ext_sales_price#33, d_year#37, d_qoy#38, ca_county#41] +Keys [3]: [ca_county#41, d_qoy#38, d_year#37] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [1]: [sum#42] +Results [4]: [ca_county#41, d_qoy#38, d_year#37, sum#43] (50) Exchange -Input [4]: [ca_county#42, d_qoy#39, d_year#38, sum#44] -Arguments: hashpartitioning(ca_county#42, d_qoy#39, d_year#38, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [4]: [ca_county#41, d_qoy#38, d_year#37, sum#43] +Arguments: hashpartitioning(ca_county#41, d_qoy#38, d_year#37, 5), ENSURE_REQUIREMENTS, [id=#44] (51) HashAggregate [codegen id : 20] -Input [4]: [ca_county#42, d_qoy#39, d_year#38, sum#44] -Keys [3]: [ca_county#42, d_qoy#39, d_year#38] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#46] -Results [3]: [ca_county#42, d_year#38, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2) AS store_sales#47] +Input [4]: [ca_county#41, d_qoy#38, d_year#37, sum#43] +Keys [3]: [ca_county#41, d_qoy#38, d_year#37] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#33))#15] +Results [3]: [ca_county#41, d_year#37, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#15,17,2) AS store_sales#45] (52) BroadcastExchange -Input [3]: [ca_county#42, d_year#38, store_sales#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [3]: [ca_county#41, d_year#37, store_sales#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (53) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#42] +Right keys [1]: [ca_county#41] Join condition: None (54) Project [codegen id : 42] -Output [5]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47] -Input [6]: [ca_county#10, store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47] +Output [5]: [store_sales#16, store_sales#30, ca_county#41, d_year#37, store_sales#45] +Input [6]: [ca_county#10, store_sales#16, store_sales#30, ca_county#41, d_year#37, store_sales#45] (55) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] (57) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_bill_addr_sk#49) +Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] +Condition : isnotnull(ws_bill_addr_sk#47) (58) ReusedExchange [Reuses operator id: 122] -Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] +Output [3]: [d_date_sk#50, d_year#51, d_qoy#52] (59) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#52] +Left keys [1]: [ws_sold_date_sk#49] +Right keys [1]: [d_date_sk#50] Join condition: None (60) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51, d_date_sk#52, d_year#53, d_qoy#54] +Output [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#51, d_qoy#52] +Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49, d_date_sk#50, d_year#51, d_qoy#52] (61) Exchange -Input [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] -Arguments: hashpartitioning(ws_bill_addr_sk#49, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#51, d_qoy#52] +Arguments: hashpartitioning(ws_bill_addr_sk#47, 5), ENSURE_REQUIREMENTS, [id=#53] (62) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54] -Arguments: [ws_bill_addr_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#51, d_qoy#52] +Arguments: [ws_bill_addr_sk#47 ASC NULLS FIRST], false, 0 (63) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#56, ca_county#57] +Output [2]: [ca_address_sk#54, ca_county#55] (64) Sort [codegen id : 25] -Input [2]: [ca_address_sk#56, ca_county#57] -Arguments: [ca_address_sk#56 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#54, ca_county#55] +Arguments: [ca_address_sk#54 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#49] -Right keys [1]: [ca_address_sk#56] +Left keys [1]: [ws_bill_addr_sk#47] +Right keys [1]: [ca_address_sk#54] Join condition: None (66) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#57] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#56, ca_county#57] +Output [4]: [ws_ext_sales_price#48, d_year#51, d_qoy#52, ca_county#55] +Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#51, d_qoy#52, ca_address_sk#54, ca_county#55] (67) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#57] -Keys [3]: [ca_county#57, d_qoy#54, d_year#53] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum#58] -Results [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] +Input [4]: [ws_ext_sales_price#48, d_year#51, d_qoy#52, ca_county#55] +Keys [3]: [ca_county#55, d_qoy#52, d_year#51] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#48))] +Aggregate Attributes [1]: [sum#56] +Results [4]: [ca_county#55, d_qoy#52, d_year#51, sum#57] (68) Exchange -Input [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] -Arguments: hashpartitioning(ca_county#57, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [4]: [ca_county#55, d_qoy#52, d_year#51, sum#57] +Arguments: hashpartitioning(ca_county#55, d_qoy#52, d_year#51, 5), ENSURE_REQUIREMENTS, [id=#58] (69) HashAggregate [codegen id : 41] -Input [4]: [ca_county#57, d_qoy#54, d_year#53, sum#59] -Keys [3]: [ca_county#57, d_qoy#54, d_year#53] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#50))#61] -Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#61,17,2) AS web_sales#62] +Input [4]: [ca_county#55, d_qoy#52, d_year#51, sum#57] +Keys [3]: [ca_county#55, d_qoy#52, d_year#51] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#48))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#48))#59] +Results [2]: [ca_county#55, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#48))#59,17,2) AS web_sales#60] (70) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (71) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] (72) Filter [codegen id : 28] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_addr_sk#63) +Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_addr_sk#61) (73) ReusedExchange [Reuses operator id: 114] -Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] +Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] (74) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (75) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#66, d_year#67, d_qoy#68] +Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] +Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] (76) Exchange -Input [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] -Arguments: hashpartitioning(ws_bill_addr_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] +Arguments: hashpartitioning(ws_bill_addr_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (77) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68] -Arguments: [ws_bill_addr_sk#63 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] +Arguments: [ws_bill_addr_sk#61 ASC NULLS FIRST], false, 0 (78) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#70, ca_county#71] +Output [2]: [ca_address_sk#68, ca_county#69] (79) Sort [codegen id : 31] -Input [2]: [ca_address_sk#70, ca_county#71] -Arguments: [ca_address_sk#70 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#68, ca_county#69] +Arguments: [ca_address_sk#68 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#70] +Left keys [1]: [ws_bill_addr_sk#61] +Right keys [1]: [ca_address_sk#68] Join condition: None (81) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#71] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#70, ca_county#71] +Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#69] +Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#68, ca_county#69] (82) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#71] -Keys [3]: [ca_county#71, d_qoy#68, d_year#67] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum#72] -Results [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] +Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#69] +Keys [3]: [ca_county#69, d_qoy#66, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] +Aggregate Attributes [1]: [sum#70] +Results [4]: [ca_county#69, d_qoy#66, d_year#65, sum#71] (83) Exchange -Input [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] -Arguments: hashpartitioning(ca_county#71, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [4]: [ca_county#69, d_qoy#66, d_year#65, sum#71] +Arguments: hashpartitioning(ca_county#69, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#72] (84) HashAggregate [codegen id : 33] -Input [4]: [ca_county#71, d_qoy#68, d_year#67, sum#73] -Keys [3]: [ca_county#71, d_qoy#68, d_year#67] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#75] -Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#75,17,2) AS web_sales#76] +Input [4]: [ca_county#69, d_qoy#66, d_year#65, sum#71] +Keys [3]: [ca_county#69, d_qoy#66, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#62))#59] +Results [2]: [ca_county#69, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#59,17,2) AS web_sales#73] (85) BroadcastExchange -Input [2]: [ca_county#71, web_sales#76] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#77] +Input [2]: [ca_county#69, web_sales#73] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] (86) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#57] -Right keys [1]: [ca_county#71] +Left keys [1]: [ca_county#55] +Right keys [1]: [ca_county#69] Join condition: None (87) Project [codegen id : 41] -Output [3]: [ca_county#57, web_sales#62, web_sales#76] -Input [4]: [ca_county#57, web_sales#62, ca_county#71, web_sales#76] +Output [3]: [ca_county#55, web_sales#60, web_sales#73] +Input [4]: [ca_county#55, web_sales#60, ca_county#69, web_sales#73] (88) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] +Output [3]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, ws_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (89) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] +Input [3]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, ws_sold_date_sk#77] (90) Filter [codegen id : 35] -Input [3]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_bill_addr_sk#78) +Input [3]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, ws_sold_date_sk#77] +Condition : isnotnull(ws_bill_addr_sk#75) (91) ReusedExchange [Reuses operator id: 118] -Output [3]: [d_date_sk#81, d_year#82, d_qoy#83] +Output [3]: [d_date_sk#78, d_year#79, d_qoy#80] (92) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ws_sold_date_sk#77] +Right keys [1]: [d_date_sk#78] Join condition: None (93) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] -Input [6]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82, d_qoy#83] +Output [4]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, d_year#79, d_qoy#80] +Input [6]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, ws_sold_date_sk#77, d_date_sk#78, d_year#79, d_qoy#80] (94) Exchange -Input [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] -Arguments: hashpartitioning(ws_bill_addr_sk#78, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [4]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, d_year#79, d_qoy#80] +Arguments: hashpartitioning(ws_bill_addr_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (95) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83] -Arguments: [ws_bill_addr_sk#78 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, d_year#79, d_qoy#80] +Arguments: [ws_bill_addr_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#85, ca_county#86] +Output [2]: [ca_address_sk#82, ca_county#83] (97) Sort [codegen id : 38] -Input [2]: [ca_address_sk#85, ca_county#86] -Arguments: [ca_address_sk#85 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#82, ca_county#83] +Arguments: [ca_address_sk#82 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#78] -Right keys [1]: [ca_address_sk#85] +Left keys [1]: [ws_bill_addr_sk#75] +Right keys [1]: [ca_address_sk#82] Join condition: None (99) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_county#86] -Input [6]: [ws_bill_addr_sk#78, ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_address_sk#85, ca_county#86] +Output [4]: [ws_ext_sales_price#76, d_year#79, d_qoy#80, ca_county#83] +Input [6]: [ws_bill_addr_sk#75, ws_ext_sales_price#76, d_year#79, d_qoy#80, ca_address_sk#82, ca_county#83] (100) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#79, d_year#82, d_qoy#83, ca_county#86] -Keys [3]: [ca_county#86, d_qoy#83, d_year#82] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#79))] -Aggregate Attributes [1]: [sum#87] -Results [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] +Input [4]: [ws_ext_sales_price#76, d_year#79, d_qoy#80, ca_county#83] +Keys [3]: [ca_county#83, d_qoy#80, d_year#79] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#76))] +Aggregate Attributes [1]: [sum#84] +Results [4]: [ca_county#83, d_qoy#80, d_year#79, sum#85] (101) Exchange -Input [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] -Arguments: hashpartitioning(ca_county#86, d_qoy#83, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [ca_county#83, d_qoy#80, d_year#79, sum#85] +Arguments: hashpartitioning(ca_county#83, d_qoy#80, d_year#79, 5), ENSURE_REQUIREMENTS, [id=#86] (102) HashAggregate [codegen id : 40] -Input [4]: [ca_county#86, d_qoy#83, d_year#82, sum#88] -Keys [3]: [ca_county#86, d_qoy#83, d_year#82] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#79))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#79))#90] -Results [2]: [ca_county#86, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#90,17,2) AS web_sales#91] +Input [4]: [ca_county#83, d_qoy#80, d_year#79, sum#85] +Keys [3]: [ca_county#83, d_qoy#80, d_year#79] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#76))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#76))#59] +Results [2]: [ca_county#83, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#76))#59,17,2) AS web_sales#87] (103) BroadcastExchange -Input [2]: [ca_county#86, web_sales#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#92] +Input [2]: [ca_county#83, web_sales#87] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#88] (104) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#57] -Right keys [1]: [ca_county#86] +Left keys [1]: [ca_county#55] +Right keys [1]: [ca_county#83] Join condition: None (105) Project [codegen id : 41] -Output [4]: [ca_county#57, web_sales#62, web_sales#76, web_sales#91] -Input [5]: [ca_county#57, web_sales#62, web_sales#76, ca_county#86, web_sales#91] +Output [4]: [ca_county#55, web_sales#60, web_sales#73, web_sales#87] +Input [5]: [ca_county#55, web_sales#60, web_sales#73, ca_county#83, web_sales#87] (106) BroadcastExchange -Input [4]: [ca_county#57, web_sales#62, web_sales#76, web_sales#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#93] +Input [4]: [ca_county#55, web_sales#60, web_sales#73, web_sales#87] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] (107) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#42] -Right keys [1]: [ca_county#57] -Join condition: ((CASE WHEN (web_sales#62 > 0.00) THEN CheckOverflow((promote_precision(web_sales#76) / promote_precision(web_sales#62)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#47 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#76 > 0.00) THEN CheckOverflow((promote_precision(web_sales#91) / promote_precision(web_sales#76)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END)) +Left keys [1]: [ca_county#41] +Right keys [1]: [ca_county#55] +Join condition: ((CASE WHEN (web_sales#60 > 0.00) THEN CheckOverflow((promote_precision(web_sales#73) / promote_precision(web_sales#60)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#45 > 0.00) THEN CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#45)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#73 > 0.00) THEN CheckOverflow((promote_precision(web_sales#87) / promote_precision(web_sales#73)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END)) (108) Project [codegen id : 42] -Output [6]: [ca_county#42, d_year#38, CheckOverflow((promote_precision(web_sales#76) / promote_precision(web_sales#62)), DecimalType(37,20), true) AS web_q1_q2_increase#94, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#47)), DecimalType(37,20), true) AS store_q1_q2_increase#95, CheckOverflow((promote_precision(web_sales#91) / promote_precision(web_sales#76)), DecimalType(37,20), true) AS web_q2_q3_increase#96, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#97] -Input [9]: [store_sales#16, store_sales#31, ca_county#42, d_year#38, store_sales#47, ca_county#57, web_sales#62, web_sales#76, web_sales#91] +Output [6]: [ca_county#41, d_year#37, CheckOverflow((promote_precision(web_sales#73) / promote_precision(web_sales#60)), DecimalType(37,20), true) AS web_q1_q2_increase#90, CheckOverflow((promote_precision(store_sales#16) / promote_precision(store_sales#45)), DecimalType(37,20), true) AS store_q1_q2_increase#91, CheckOverflow((promote_precision(web_sales#87) / promote_precision(web_sales#73)), DecimalType(37,20), true) AS web_q2_q3_increase#92, CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q2_q3_increase#93] +Input [9]: [store_sales#16, store_sales#30, ca_county#41, d_year#37, store_sales#45, ca_county#55, web_sales#60, web_sales#73, web_sales#87] (109) Exchange -Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#94, store_q1_q2_increase#95, web_q2_q3_increase#96, store_q2_q3_increase#97] -Arguments: rangepartitioning(ca_county#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [6]: [ca_county#41, d_year#37, web_q1_q2_increase#90, store_q1_q2_increase#91, web_q2_q3_increase#92, store_q2_q3_increase#93] +Arguments: rangepartitioning(ca_county#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#94] (110) Sort [codegen id : 43] -Input [6]: [ca_county#42, d_year#38, web_q1_q2_increase#94, store_q1_q2_increase#95, web_q2_q3_increase#96, store_q2_q3_increase#97] -Arguments: [ca_county#42 ASC NULLS FIRST], true, 0 +Input [6]: [ca_county#41, d_year#37, web_q1_q2_increase#90, store_q1_q2_increase#91, web_q2_q3_increase#92, store_q2_q3_increase#93] +Arguments: [ca_county#41 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -638,7 +638,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A (114) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 BroadcastExchange (118) @@ -663,9 +663,9 @@ Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 3) (118) BroadcastExchange Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] -Subquery:3 Hosting operator id = 37 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:3 Hosting operator id = 37 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (122) +- * Filter (121) +- * ColumnarToRow (120) @@ -673,27 +673,27 @@ BroadcastExchange (122) (119) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Output [3]: [d_date_sk#36, d_year#37, d_qoy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Input [3]: [d_date_sk#36, d_year#37, d_qoy#38] (121) Filter [codegen id : 1] -Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Condition : ((((isnotnull(d_qoy#39) AND isnotnull(d_year#38)) AND (d_qoy#39 = 1)) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#36, d_year#37, d_qoy#38] +Condition : ((((isnotnull(d_qoy#38) AND isnotnull(d_year#37)) AND (d_qoy#38 = 1)) AND (d_year#37 = 2000)) AND isnotnull(d_date_sk#36)) (122) BroadcastExchange -Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#101] +Input [3]: [d_date_sk#36, d_year#37, d_qoy#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#97] -Subquery:4 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 55 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#35 -Subquery:5 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#20 +Subquery:6 Hosting operator id = 88 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#20 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 9f758c9c4cf6b..472cd0bd2f382 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -217,12 +217,12 @@ Arguments: hashpartitioning(ca_county#24, d_qoy#22, d_year#21, 5), ENSURE_REQUIR Input [4]: [ca_county#24, d_qoy#22, d_year#21, sum#26] Keys [3]: [ca_county#24, d_qoy#22, d_year#21] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#17))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#17))#28] -Results [2]: [ca_county#24, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#17))#28,17,2) AS store_sales#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#17))#14] +Results [2]: [ca_county#24, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#17))#14,17,2) AS store_sales#28] (28) BroadcastExchange -Input [2]: [ca_county#24, store_sales#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [2]: [ca_county#24, store_sales#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#29] (29) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] @@ -230,287 +230,287 @@ Right keys [1]: [ca_county#24] Join condition: None (30) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] +Output [3]: [ss_addr_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] +Input [3]: [ss_addr_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] (32) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33] -Condition : isnotnull(ss_addr_sk#31) +Input [3]: [ss_addr_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_addr_sk#30) (33) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#35, d_year#36, d_qoy#37] +Output [3]: [d_date_sk#34, d_year#35, d_qoy#36] (34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#34] Join condition: None (35) Project [codegen id : 10] -Output [4]: [ss_addr_sk#31, ss_ext_sales_price#32, d_year#36, d_qoy#37] -Input [6]: [ss_addr_sk#31, ss_ext_sales_price#32, ss_sold_date_sk#33, d_date_sk#35, d_year#36, d_qoy#37] +Output [4]: [ss_addr_sk#30, ss_ext_sales_price#31, d_year#35, d_qoy#36] +Input [6]: [ss_addr_sk#30, ss_ext_sales_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35, d_qoy#36] (36) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#38, ca_county#39] +Output [2]: [ca_address_sk#37, ca_county#38] (37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#31] -Right keys [1]: [ca_address_sk#38] +Left keys [1]: [ss_addr_sk#30] +Right keys [1]: [ca_address_sk#37] Join condition: None (38) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#32, d_year#36, d_qoy#37, ca_county#39] -Input [6]: [ss_addr_sk#31, ss_ext_sales_price#32, d_year#36, d_qoy#37, ca_address_sk#38, ca_county#39] +Output [4]: [ss_ext_sales_price#31, d_year#35, d_qoy#36, ca_county#38] +Input [6]: [ss_addr_sk#30, ss_ext_sales_price#31, d_year#35, d_qoy#36, ca_address_sk#37, ca_county#38] (39) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#32, d_year#36, d_qoy#37, ca_county#39] -Keys [3]: [ca_county#39, d_qoy#37, d_year#36] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#40] -Results [4]: [ca_county#39, d_qoy#37, d_year#36, sum#41] +Input [4]: [ss_ext_sales_price#31, d_year#35, d_qoy#36, ca_county#38] +Keys [3]: [ca_county#38, d_qoy#36, d_year#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#31))] +Aggregate Attributes [1]: [sum#39] +Results [4]: [ca_county#38, d_qoy#36, d_year#35, sum#40] (40) Exchange -Input [4]: [ca_county#39, d_qoy#37, d_year#36, sum#41] -Arguments: hashpartitioning(ca_county#39, d_qoy#37, d_year#36, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [4]: [ca_county#38, d_qoy#36, d_year#35, sum#40] +Arguments: hashpartitioning(ca_county#38, d_qoy#36, d_year#35, 5), ENSURE_REQUIREMENTS, [id=#41] (41) HashAggregate [codegen id : 11] -Input [4]: [ca_county#39, d_qoy#37, d_year#36, sum#41] -Keys [3]: [ca_county#39, d_qoy#37, d_year#36] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#32))#43] -Results [2]: [ca_county#39, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#32))#43,17,2) AS store_sales#44] +Input [4]: [ca_county#38, d_qoy#36, d_year#35, sum#40] +Keys [3]: [ca_county#38, d_qoy#36, d_year#35] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#31))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#31))#14] +Results [2]: [ca_county#38, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#14,17,2) AS store_sales#42] (42) BroadcastExchange -Input [2]: [ca_county#39, store_sales#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +Input [2]: [ca_county#38, store_sales#42] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] (43) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#24] -Right keys [1]: [ca_county#39] +Right keys [1]: [ca_county#38] Join condition: None (44) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44] -Input [7]: [ca_county#9, d_year#6, store_sales#15, ca_county#24, store_sales#29, ca_county#39, store_sales#44] +Output [5]: [ca_county#9, d_year#6, store_sales#15, store_sales#28, store_sales#42] +Input [7]: [ca_county#9, d_year#6, store_sales#15, ca_county#24, store_sales#28, ca_county#38, store_sales#42] (45) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Output [3]: [ws_bill_addr_sk#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#44, ws_ext_sales_price#45, ws_sold_date_sk#46] (47) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] -Condition : isnotnull(ws_bill_addr_sk#46) +Input [3]: [ws_bill_addr_sk#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_addr_sk#44) (48) ReusedExchange [Reuses operator id: 94] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Output [3]: [d_date_sk#47, d_year#48, d_qoy#49] (49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#47] Join condition: None (50) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#49, d_year#50, d_qoy#51] +Output [4]: [ws_bill_addr_sk#44, ws_ext_sales_price#45, d_year#48, d_qoy#49] +Input [6]: [ws_bill_addr_sk#44, ws_ext_sales_price#45, ws_sold_date_sk#46, d_date_sk#47, d_year#48, d_qoy#49] (51) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] +Output [2]: [ca_address_sk#50, ca_county#51] (52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#52] +Left keys [1]: [ws_bill_addr_sk#44] +Right keys [1]: [ca_address_sk#50] Join condition: None (53) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#53] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] +Output [4]: [ws_ext_sales_price#45, d_year#48, d_qoy#49, ca_county#51] +Input [6]: [ws_bill_addr_sk#44, ws_ext_sales_price#45, d_year#48, d_qoy#49, ca_address_sk#50, ca_county#51] (54) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#54] -Results [4]: [ca_county#53, d_qoy#51, d_year#50, sum#55] +Input [4]: [ws_ext_sales_price#45, d_year#48, d_qoy#49, ca_county#51] +Keys [3]: [ca_county#51, d_qoy#49, d_year#48] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#45))] +Aggregate Attributes [1]: [sum#52] +Results [4]: [ca_county#51, d_qoy#49, d_year#48, sum#53] (55) Exchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#55] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [ca_county#51, d_qoy#49, d_year#48, sum#53] +Arguments: hashpartitioning(ca_county#51, d_qoy#49, d_year#48, 5), ENSURE_REQUIREMENTS, [id=#54] (56) HashAggregate [codegen id : 15] -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#55] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#57] -Results [2]: [ca_county#53, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#57,17,2) AS web_sales#58] +Input [4]: [ca_county#51, d_qoy#49, d_year#48, sum#53] +Keys [3]: [ca_county#51, d_qoy#49, d_year#48] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#45))#55] +Results [2]: [ca_county#51, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#45))#55,17,2) AS web_sales#56] (57) BroadcastExchange -Input [2]: [ca_county#53, web_sales#58] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#59] +Input [2]: [ca_county#51, web_sales#56] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#57] (58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#53] +Right keys [1]: [ca_county#51] Join condition: None (59) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] +Output [3]: [ws_bill_addr_sk#58, ws_ext_sales_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#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] +Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] (61) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_addr_sk#60) +Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_addr_sk#58) (62) ReusedExchange [Reuses operator id: 98] -Output [3]: [d_date_sk#63, d_year#64, d_qoy#65] +Output [3]: [d_date_sk#61, d_year#62, d_qoy#63] (63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#61] Join condition: None (64) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, d_year#64, d_qoy#65] -Input [6]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, ws_sold_date_sk#62, d_date_sk#63, d_year#64, d_qoy#65] +Output [4]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#62, d_qoy#63] +Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60, d_date_sk#61, d_year#62, d_qoy#63] (65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#66, ca_county#67] +Output [2]: [ca_address_sk#64, ca_county#65] (66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#60] -Right keys [1]: [ca_address_sk#66] +Left keys [1]: [ws_bill_addr_sk#58] +Right keys [1]: [ca_address_sk#64] Join condition: None (67) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_county#67] -Input [6]: [ws_bill_addr_sk#60, ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_address_sk#66, ca_county#67] +Output [4]: [ws_ext_sales_price#59, d_year#62, d_qoy#63, ca_county#65] +Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#62, d_qoy#63, ca_address_sk#64, ca_county#65] (68) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#61, d_year#64, d_qoy#65, ca_county#67] -Keys [3]: [ca_county#67, d_qoy#65, d_year#64] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#61))] -Aggregate Attributes [1]: [sum#68] -Results [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] +Input [4]: [ws_ext_sales_price#59, d_year#62, d_qoy#63, ca_county#65] +Keys [3]: [ca_county#65, d_qoy#63, d_year#62] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#59))] +Aggregate Attributes [1]: [sum#66] +Results [4]: [ca_county#65, d_qoy#63, d_year#62, sum#67] (69) Exchange -Input [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] -Arguments: hashpartitioning(ca_county#67, d_qoy#65, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ca_county#65, d_qoy#63, d_year#62, sum#67] +Arguments: hashpartitioning(ca_county#65, d_qoy#63, d_year#62, 5), ENSURE_REQUIREMENTS, [id=#68] (70) HashAggregate [codegen id : 19] -Input [4]: [ca_county#67, d_qoy#65, d_year#64, sum#69] -Keys [3]: [ca_county#67, d_qoy#65, d_year#64] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#61))#71] -Results [2]: [ca_county#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#61))#71,17,2) AS web_sales#72] +Input [4]: [ca_county#65, d_qoy#63, d_year#62, sum#67] +Keys [3]: [ca_county#65, d_qoy#63, d_year#62] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#59))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#59))#55] +Results [2]: [ca_county#65, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#55,17,2) AS web_sales#69] (71) BroadcastExchange -Input [2]: [ca_county#67, web_sales#72] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] +Input [2]: [ca_county#65, web_sales#69] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#70] (72) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#53] -Right keys [1]: [ca_county#67] -Join condition: (CASE WHEN (web_sales#58 > 0.00) THEN CheckOverflow((promote_precision(web_sales#72) / promote_precision(web_sales#58)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#51] +Right keys [1]: [ca_county#65] +Join condition: (CASE WHEN (web_sales#56 > 0.00) THEN CheckOverflow((promote_precision(web_sales#69) / promote_precision(web_sales#56)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#15 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#15)), DecimalType(37,20), true) ELSE null END) (73) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#72] -Input [9]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, ca_county#67, web_sales#72] +Output [8]: [ca_county#9, d_year#6, store_sales#15, store_sales#28, store_sales#42, ca_county#51, web_sales#56, web_sales#69] +Input [9]: [ca_county#9, d_year#6, store_sales#15, store_sales#28, store_sales#42, ca_county#51, web_sales#56, ca_county#65, web_sales#69] (74) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] +Output [3]: [ws_bill_addr_sk#71, ws_ext_sales_price#72, ws_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#33)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] +Input [3]: [ws_bill_addr_sk#71, ws_ext_sales_price#72, ws_sold_date_sk#73] (76) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_bill_addr_sk#74) +Input [3]: [ws_bill_addr_sk#71, ws_ext_sales_price#72, ws_sold_date_sk#73] +Condition : isnotnull(ws_bill_addr_sk#71) (77) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#77, d_year#78, d_qoy#79] +Output [3]: [d_date_sk#74, d_year#75, d_qoy#76] (78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#76] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ws_sold_date_sk#73] +Right keys [1]: [d_date_sk#74] Join condition: None (79) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, d_year#78, d_qoy#79] -Input [6]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78, d_qoy#79] +Output [4]: [ws_bill_addr_sk#71, ws_ext_sales_price#72, d_year#75, d_qoy#76] +Input [6]: [ws_bill_addr_sk#71, ws_ext_sales_price#72, ws_sold_date_sk#73, d_date_sk#74, d_year#75, d_qoy#76] (80) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#80, ca_county#81] +Output [2]: [ca_address_sk#77, ca_county#78] (81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#74] -Right keys [1]: [ca_address_sk#80] +Left keys [1]: [ws_bill_addr_sk#71] +Right keys [1]: [ca_address_sk#77] Join condition: None (82) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_county#81] -Input [6]: [ws_bill_addr_sk#74, ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_address_sk#80, ca_county#81] +Output [4]: [ws_ext_sales_price#72, d_year#75, d_qoy#76, ca_county#78] +Input [6]: [ws_bill_addr_sk#71, ws_ext_sales_price#72, d_year#75, d_qoy#76, ca_address_sk#77, ca_county#78] (83) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#75, d_year#78, d_qoy#79, ca_county#81] -Keys [3]: [ca_county#81, d_qoy#79, d_year#78] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#75))] -Aggregate Attributes [1]: [sum#82] -Results [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] +Input [4]: [ws_ext_sales_price#72, d_year#75, d_qoy#76, ca_county#78] +Keys [3]: [ca_county#78, d_qoy#76, d_year#75] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#72))] +Aggregate Attributes [1]: [sum#79] +Results [4]: [ca_county#78, d_qoy#76, d_year#75, sum#80] (84) Exchange -Input [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] -Arguments: hashpartitioning(ca_county#81, d_qoy#79, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [4]: [ca_county#78, d_qoy#76, d_year#75, sum#80] +Arguments: hashpartitioning(ca_county#78, d_qoy#76, d_year#75, 5), ENSURE_REQUIREMENTS, [id=#81] (85) HashAggregate [codegen id : 23] -Input [4]: [ca_county#81, d_qoy#79, d_year#78, sum#83] -Keys [3]: [ca_county#81, d_qoy#79, d_year#78] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#75))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#75))#85] -Results [2]: [ca_county#81, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#75))#85,17,2) AS web_sales#86] +Input [4]: [ca_county#78, d_qoy#76, d_year#75, sum#80] +Keys [3]: [ca_county#78, d_qoy#76, d_year#75] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#72))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#72))#55] +Results [2]: [ca_county#78, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#55,17,2) AS web_sales#82] (86) BroadcastExchange -Input [2]: [ca_county#81, web_sales#86] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#87] +Input [2]: [ca_county#78, web_sales#82] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#83] (87) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#53] -Right keys [1]: [ca_county#81] -Join condition: (CASE WHEN (web_sales#72 > 0.00) THEN CheckOverflow((promote_precision(web_sales#86) / promote_precision(web_sales#72)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#29 > 0.00) THEN CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#51] +Right keys [1]: [ca_county#78] +Join condition: (CASE WHEN (web_sales#69 > 0.00) THEN CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#69)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#42) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END) (88) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#72) / promote_precision(web_sales#58)), DecimalType(37,20), true) AS web_q1_q2_increase#88, CheckOverflow((promote_precision(store_sales#29) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#89, CheckOverflow((promote_precision(web_sales#86) / promote_precision(web_sales#72)), DecimalType(37,20), true) AS web_q2_q3_increase#90, CheckOverflow((promote_precision(store_sales#44) / promote_precision(store_sales#29)), DecimalType(37,20), true) AS store_q2_q3_increase#91] -Input [10]: [ca_county#9, d_year#6, store_sales#15, store_sales#29, store_sales#44, ca_county#53, web_sales#58, web_sales#72, ca_county#81, web_sales#86] +Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#69) / promote_precision(web_sales#56)), DecimalType(37,20), true) AS web_q1_q2_increase#84, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#15)), DecimalType(37,20), true) AS store_q1_q2_increase#85, CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#69)), DecimalType(37,20), true) AS web_q2_q3_increase#86, CheckOverflow((promote_precision(store_sales#42) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#87] +Input [10]: [ca_county#9, d_year#6, store_sales#15, store_sales#28, store_sales#42, ca_county#51, web_sales#56, web_sales#69, ca_county#78, web_sales#82] (89) Exchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#88, store_q1_q2_increase#89, web_q2_q3_increase#90, store_q2_q3_increase#91] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#84, store_q1_q2_increase#85, web_q2_q3_increase#86, store_q2_q3_increase#87] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#88] (90) Sort [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#88, store_q1_q2_increase#89, web_q2_q3_increase#90, store_q2_q3_increase#91] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#84, store_q1_q2_increase#85, web_q2_q3_increase#86, store_q2_q3_increase#87] Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -538,7 +538,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) A (94) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#89] Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (98) @@ -563,9 +563,9 @@ Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2) (98) BroadcastExchange Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#34 +Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (102) +- * Filter (101) +- * ColumnarToRow (100) @@ -573,27 +573,27 @@ BroadcastExchange (102) (99) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_qoy#37] +Output [3]: [d_date_sk#34, d_year#35, d_qoy#36] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] +Input [3]: [d_date_sk#34, d_year#35, d_qoy#36] (101) Filter [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] -Condition : ((((isnotnull(d_qoy#37) AND isnotnull(d_year#36)) AND (d_qoy#37 = 3)) AND (d_year#36 = 2000)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#34, d_year#35, d_qoy#36] +Condition : ((((isnotnull(d_qoy#36) AND isnotnull(d_year#35)) AND (d_qoy#36 = 3)) AND (d_year#35 = 2000)) AND isnotnull(d_date_sk#34)) (102) BroadcastExchange -Input [3]: [d_date_sk#35, d_year#36, d_qoy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Input [3]: [d_date_sk#34, d_year#35, d_qoy#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#91] -Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#19 +Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#19 -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#34 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#73 IN dynamicpruning#33 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index 2850c59dbdbee..cd5f54fed82a5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -229,23 +229,23 @@ Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37 Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#36, n#45, avg#46, m2#47, sum#48, count#49] Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#36] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#32 as double)), avg(inv_quantity_on_hand#32)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#32 as double))#51, avg(inv_quantity_on_hand#32)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stddev_samp(cast(inv_quantity_on_hand#32 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#32)#52 AS mean#54] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#32 as double))#24, avg(inv_quantity_on_hand#32)#25] +Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stddev_samp(cast(inv_quantity_on_hand#32 as double))#24 AS stdev#26, avg(inv_quantity_on_hand#32)#25 AS mean#27] (41) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#53, mean#54] -Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#26, mean#27] +Condition : ((isnotnull(mean#27) AND isnotnull(stdev#26)) AND (NOT (mean#27 = 0.0) AND ((stdev#26 / mean#27) > 1.0))) (42) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#27 AS mean#51, CASE WHEN (mean#27 = 0.0) THEN null ELSE (stdev#26 / mean#27) END AS cov#52] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#26, mean#27] (43) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] +Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#53] (44) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 13] @@ -254,12 +254,12 @@ Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] Join condition: None (46) Exchange -Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] +Arguments: rangepartitioning(w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#54] (47) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] -Arguments: [w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] +Arguments: [w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -272,26 +272,26 @@ BroadcastExchange (52) (48) Scan parquet default.date_dim -Output [3]: [d_date_sk#6, d_year#58, d_moy#7] +Output [3]: [d_date_sk#6, d_year#55, d_moy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#58, d_moy#7] +Input [3]: [d_date_sk#6, d_year#55, d_moy#7] (50) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#58, d_moy#7] -Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#7)) AND (d_year#58 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#55, d_moy#7] +Condition : ((((isnotnull(d_year#55) AND isnotnull(d_moy#7)) AND (d_year#55 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) (51) Project [codegen id : 1] Output [2]: [d_date_sk#6, d_moy#7] -Input [3]: [d_date_sk#6, d_year#58, d_moy#7] +Input [3]: [d_date_sk#6, d_year#55, d_moy#7] (52) BroadcastExchange Input [2]: [d_date_sk#6, d_moy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#33 IN dynamicpruning#34 BroadcastExchange (57) @@ -302,25 +302,25 @@ BroadcastExchange (57) (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#60, d_moy#36] +Output [3]: [d_date_sk#35, d_year#57, d_moy#36] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#60, d_moy#36] +Input [3]: [d_date_sk#35, d_year#57, d_moy#36] (55) Filter [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#60, d_moy#36] -Condition : ((((isnotnull(d_year#60) AND isnotnull(d_moy#36)) AND (d_year#60 = 2001)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#35, d_year#57, d_moy#36] +Condition : ((((isnotnull(d_year#57) AND isnotnull(d_moy#36)) AND (d_year#57 = 2001)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#35)) (56) Project [codegen id : 1] Output [2]: [d_date_sk#35, d_moy#36] -Input [3]: [d_date_sk#35, d_year#60, d_moy#36] +Input [3]: [d_date_sk#35, d_year#57, d_moy#36] (57) BroadcastExchange Input [2]: [d_date_sk#35, d_moy#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index c117b681c8eb4..eb876b023bcec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -218,20 +218,20 @@ Arguments: hashpartitioning(w_warehouse_name#36, w_warehouse_sk#35, i_item_sk#34 Input [9]: [w_warehouse_name#36, w_warehouse_sk#35, i_item_sk#34, d_moy#38, n#44, avg#45, m2#46, sum#47, count#48] Keys [4]: [w_warehouse_name#36, w_warehouse_sk#35, i_item_sk#34, d_moy#38] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#31 as double)), avg(inv_quantity_on_hand#31)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#31 as double))#50, avg(inv_quantity_on_hand#31)#51] -Results [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stddev_samp(cast(inv_quantity_on_hand#31 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#31)#51 AS mean#53] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#31 as double))#24, avg(inv_quantity_on_hand#31)#25] +Results [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stddev_samp(cast(inv_quantity_on_hand#31 as double))#24 AS stdev#26, avg(inv_quantity_on_hand#31)#25 AS mean#27] (39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#52, mean#53] -Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) +Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#26, mean#27] +Condition : ((isnotnull(mean#27) AND isnotnull(stdev#26)) AND (NOT (mean#27 = 0.0) AND ((stdev#26 / mean#27) > 1.0))) (40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#27 AS mean#50, CASE WHEN (mean#27 = 0.0) THEN null ELSE (stdev#26 / mean#27) END AS cov#51] +Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#26, mean#27] (41) BroadcastExchange -Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, cov#54] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#55] +Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#50, cov#51] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#52] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] @@ -239,12 +239,12 @@ Right keys [2]: [i_item_sk#34, w_warehouse_sk#35] Join condition: None (43) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#50, cov#51] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#50 ASC NULLS FIRST, cov#51 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#53] (44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#50, cov#51] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#50 ASC NULLS FIRST, cov#51 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -257,26 +257,26 @@ BroadcastExchange (49) (45) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#57, d_moy#12] +Output [3]: [d_date_sk#11, d_year#54, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#57, d_moy#12] +Input [3]: [d_date_sk#11, d_year#54, d_moy#12] (47) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#57, d_moy#12] -Condition : ((((isnotnull(d_year#57) AND isnotnull(d_moy#12)) AND (d_year#57 = 2001)) AND (d_moy#12 = 1)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_year#54, d_moy#12] +Condition : ((((isnotnull(d_year#54) AND isnotnull(d_moy#12)) AND (d_year#54 = 2001)) AND (d_moy#12 = 1)) AND isnotnull(d_date_sk#11)) (48) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_moy#12] -Input [3]: [d_date_sk#11, d_year#57, d_moy#12] +Input [3]: [d_date_sk#11, d_year#54, d_moy#12] (49) BroadcastExchange Input [2]: [d_date_sk#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#32 IN dynamicpruning#33 BroadcastExchange (54) @@ -287,25 +287,25 @@ BroadcastExchange (54) (50) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#59, d_moy#38] +Output [3]: [d_date_sk#37, d_year#56, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#37, d_year#59, d_moy#38] +Input [3]: [d_date_sk#37, d_year#56, d_moy#38] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#37, d_year#59, d_moy#38] -Condition : ((((isnotnull(d_year#59) AND isnotnull(d_moy#38)) AND (d_year#59 = 2001)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#37, d_year#56, d_moy#38] +Condition : ((((isnotnull(d_year#56) AND isnotnull(d_moy#38)) AND (d_year#56 = 2001)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#37)) (53) Project [codegen id : 1] Output [2]: [d_date_sk#37, d_moy#38] -Input [3]: [d_date_sk#37, d_year#59, d_moy#38] +Input [3]: [d_date_sk#37, d_year#56, d_moy#38] (54) BroadcastExchange Input [2]: [d_date_sk#37, d_moy#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index 492698c2ed2cc..e053021e60df9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -229,23 +229,23 @@ Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37 Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#36, n#45, avg#46, m2#47, sum#48, count#49] Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#36] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#32 as double)), avg(inv_quantity_on_hand#32)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#32 as double))#51, avg(inv_quantity_on_hand#32)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stddev_samp(cast(inv_quantity_on_hand#32 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#32)#52 AS mean#54] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#32 as double))#24, avg(inv_quantity_on_hand#32)#25] +Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stddev_samp(cast(inv_quantity_on_hand#32 as double))#24 AS stdev#26, avg(inv_quantity_on_hand#32)#25 AS mean#27] (41) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#53, mean#54] -Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#26, mean#27] +Condition : ((isnotnull(mean#27) AND isnotnull(stdev#26)) AND (NOT (mean#27 = 0.0) AND ((stdev#26 / mean#27) > 1.0))) (42) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#27 AS mean#51, CASE WHEN (mean#27 = 0.0) THEN null ELSE (stdev#26 / mean#27) END AS cov#52] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, stdev#26, mean#27] (43) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] +Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#53] (44) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 13] @@ -254,12 +254,12 @@ Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] Join condition: None (46) Exchange -Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] +Arguments: rangepartitioning(w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#54] (47) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#54, cov#55] -Arguments: [w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#10, i_item_sk#8, d_moy#7, mean#27, cov#28, w_warehouse_sk#38, i_item_sk#37, d_moy#36, mean#51, cov#52] +Arguments: [w_warehouse_sk#10 ASC NULLS FIRST, i_item_sk#8 ASC NULLS FIRST, d_moy#7 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, mean#51 ASC NULLS FIRST, cov#52 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -272,26 +272,26 @@ BroadcastExchange (52) (48) Scan parquet default.date_dim -Output [3]: [d_date_sk#6, d_year#58, d_moy#7] +Output [3]: [d_date_sk#6, d_year#55, d_moy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#58, d_moy#7] +Input [3]: [d_date_sk#6, d_year#55, d_moy#7] (50) Filter [codegen id : 1] -Input [3]: [d_date_sk#6, d_year#58, d_moy#7] -Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#7)) AND (d_year#58 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#55, d_moy#7] +Condition : ((((isnotnull(d_year#55) AND isnotnull(d_moy#7)) AND (d_year#55 = 2001)) AND (d_moy#7 = 1)) AND isnotnull(d_date_sk#6)) (51) Project [codegen id : 1] Output [2]: [d_date_sk#6, d_moy#7] -Input [3]: [d_date_sk#6, d_year#58, d_moy#7] +Input [3]: [d_date_sk#6, d_year#55, d_moy#7] (52) BroadcastExchange Input [2]: [d_date_sk#6, d_moy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#33 IN dynamicpruning#34 BroadcastExchange (57) @@ -302,25 +302,25 @@ BroadcastExchange (57) (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#60, d_moy#36] +Output [3]: [d_date_sk#35, d_year#57, d_moy#36] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#60, d_moy#36] +Input [3]: [d_date_sk#35, d_year#57, d_moy#36] (55) Filter [codegen id : 1] -Input [3]: [d_date_sk#35, d_year#60, d_moy#36] -Condition : ((((isnotnull(d_year#60) AND isnotnull(d_moy#36)) AND (d_year#60 = 2001)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#35, d_year#57, d_moy#36] +Condition : ((((isnotnull(d_year#57) AND isnotnull(d_moy#36)) AND (d_year#57 = 2001)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#35)) (56) Project [codegen id : 1] Output [2]: [d_date_sk#35, d_moy#36] -Input [3]: [d_date_sk#35, d_year#60, d_moy#36] +Input [3]: [d_date_sk#35, d_year#57, d_moy#36] (57) BroadcastExchange Input [2]: [d_date_sk#35, d_moy#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index ccc0253a77e64..fef701c0e3b4a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -218,20 +218,20 @@ Arguments: hashpartitioning(w_warehouse_name#36, w_warehouse_sk#35, i_item_sk#34 Input [9]: [w_warehouse_name#36, w_warehouse_sk#35, i_item_sk#34, d_moy#38, n#44, avg#45, m2#46, sum#47, count#48] Keys [4]: [w_warehouse_name#36, w_warehouse_sk#35, i_item_sk#34, d_moy#38] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#31 as double)), avg(inv_quantity_on_hand#31)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#31 as double))#50, avg(inv_quantity_on_hand#31)#51] -Results [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stddev_samp(cast(inv_quantity_on_hand#31 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#31)#51 AS mean#53] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#31 as double))#24, avg(inv_quantity_on_hand#31)#25] +Results [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stddev_samp(cast(inv_quantity_on_hand#31 as double))#24 AS stdev#26, avg(inv_quantity_on_hand#31)#25 AS mean#27] (39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#52, mean#53] -Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) +Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#26, mean#27] +Condition : ((isnotnull(mean#27) AND isnotnull(stdev#26)) AND (NOT (mean#27 = 0.0) AND ((stdev#26 / mean#27) > 1.0))) (40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#27 AS mean#50, CASE WHEN (mean#27 = 0.0) THEN null ELSE (stdev#26 / mean#27) END AS cov#51] +Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, stdev#26, mean#27] (41) BroadcastExchange -Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, cov#54] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#55] +Input [5]: [w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#50, cov#51] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#52] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] @@ -239,12 +239,12 @@ Right keys [2]: [i_item_sk#34, w_warehouse_sk#35] Join condition: None (43) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#50, cov#51] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#50 ASC NULLS FIRST, cov#51 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#53] (44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#12, mean#27, cov#28, w_warehouse_sk#35, i_item_sk#34, d_moy#38, mean#50, cov#51] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST, mean#27 ASC NULLS FIRST, cov#28 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, mean#50 ASC NULLS FIRST, cov#51 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -257,26 +257,26 @@ BroadcastExchange (49) (45) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#57, d_moy#12] +Output [3]: [d_date_sk#11, d_year#54, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#57, d_moy#12] +Input [3]: [d_date_sk#11, d_year#54, d_moy#12] (47) Filter [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#57, d_moy#12] -Condition : ((((isnotnull(d_year#57) AND isnotnull(d_moy#12)) AND (d_year#57 = 2001)) AND (d_moy#12 = 1)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#11, d_year#54, d_moy#12] +Condition : ((((isnotnull(d_year#54) AND isnotnull(d_moy#12)) AND (d_year#54 = 2001)) AND (d_moy#12 = 1)) AND isnotnull(d_date_sk#11)) (48) Project [codegen id : 1] Output [2]: [d_date_sk#11, d_moy#12] -Input [3]: [d_date_sk#11, d_year#57, d_moy#12] +Input [3]: [d_date_sk#11, d_year#54, d_moy#12] (49) BroadcastExchange Input [2]: [d_date_sk#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#55] Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#32 IN dynamicpruning#33 BroadcastExchange (54) @@ -287,25 +287,25 @@ BroadcastExchange (54) (50) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#59, d_moy#38] +Output [3]: [d_date_sk#37, d_year#56, d_moy#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#37, d_year#59, d_moy#38] +Input [3]: [d_date_sk#37, d_year#56, d_moy#38] (52) Filter [codegen id : 1] -Input [3]: [d_date_sk#37, d_year#59, d_moy#38] -Condition : ((((isnotnull(d_year#59) AND isnotnull(d_moy#38)) AND (d_year#59 = 2001)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#37, d_year#56, d_moy#38] +Condition : ((((isnotnull(d_year#56) AND isnotnull(d_moy#38)) AND (d_year#56 = 2001)) AND (d_moy#38 = 2)) AND isnotnull(d_date_sk#37)) (53) Project [codegen id : 1] Output [2]: [d_date_sk#37, d_moy#38] -Input [3]: [d_date_sk#37, d_year#59, d_moy#38] +Input [3]: [d_date_sk#37, d_year#56, d_moy#38] (54) BroadcastExchange Input [2]: [d_date_sk#37, d_moy#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index a311bec3c313b..3b644fb8bd7a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -281,377 +281,377 @@ Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, c Input [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#37, sum#49, isEmpty#50] Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#37] Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#32 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#31 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#32 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#31 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52] -Results [8]: [c_customer_id#40 AS customer_id#53, c_first_name#41 AS customer_first_name#54, c_last_name#42 AS customer_last_name#55, c_preferred_cust_flag#43 AS customer_preferred_cust_flag#56, c_birth_country#44 AS customer_birth_country#57, c_login#45 AS customer_login#58, c_email_address#46 AS customer_email_address#59, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#32 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#31 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52 AS year_total#60] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#32 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#31 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#25] +Results [8]: [c_customer_id#40 AS customer_id#52, c_first_name#41 AS customer_first_name#53, c_last_name#42 AS customer_last_name#54, c_preferred_cust_flag#43 AS customer_preferred_cust_flag#55, c_birth_country#44 AS customer_birth_country#56, c_login#45 AS customer_login#57, c_email_address#46 AS customer_email_address#58, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#32 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#31 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#25 AS year_total#59] (37) Exchange -Input [8]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Arguments: hashpartitioning(customer_id#53, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [8]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59] +Arguments: hashpartitioning(customer_id#52, 5), ENSURE_REQUIREMENTS, [id=#60] (38) Sort [codegen id : 16] -Input [8]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Arguments: [customer_id#53 ASC NULLS FIRST], false, 0 +Input [8]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59] +Arguments: [customer_id#52 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#53] +Right keys [1]: [customer_id#52] Join condition: None (40) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Output [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_sold_date_sk#67 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 19] -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] (42) Filter [codegen id : 19] -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Condition : isnotnull(cs_bill_customer_sk#62) +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66] +Condition : isnotnull(cs_bill_customer_sk#61) (43) ReusedExchange [Reuses operator id: 122] -Output [2]: [d_date_sk#68, d_year#69] +Output [2]: [d_date_sk#67, d_year#68] (44) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#67] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#67] Join condition: None (45) Project [codegen id : 19] -Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69] -Input [8]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#68, d_year#69] +Output [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#68] +Input [8]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, cs_sold_date_sk#66, d_date_sk#67, d_year#68] (46) Exchange -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69] -Arguments: hashpartitioning(cs_bill_customer_sk#62, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#68] +Arguments: hashpartitioning(cs_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#69] (47) Sort [codegen id : 20] -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69] -Arguments: [cs_bill_customer_sk#62 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#68] +Arguments: [cs_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (48) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] (49) Sort [codegen id : 22] -Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] -Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Arguments: [c_customer_sk#70 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#62] -Right keys [1]: [c_customer_sk#71] +Left keys [1]: [cs_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#70] Join condition: None (51) Project [codegen id : 23] -Output [12]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69] -Input [14]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69, c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Output [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#68] +Input [14]: [cs_bill_customer_sk#61, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#68, c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] (52) HashAggregate [codegen id : 23] -Input [12]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69] -Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#69] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#79, isEmpty#80] -Results [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#69, sum#81, isEmpty#82] +Input [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, cs_ext_discount_amt#62, cs_ext_sales_price#63, cs_ext_wholesale_cost#64, cs_ext_list_price#65, d_year#68] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#68] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#65 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#64 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#62 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#63 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#68, sum#80, isEmpty#81] (53) Exchange -Input [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#69, sum#81, isEmpty#82] -Arguments: hashpartitioning(c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#68, sum#80, isEmpty#81] +Arguments: hashpartitioning(c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#82] (54) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#69, sum#81, isEmpty#82] -Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#69] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84] -Results [2]: [c_customer_id#72 AS customer_id#85, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84 AS year_total#86] +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#68, sum#80, isEmpty#81] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#68] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#65 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#64 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#62 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#63 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#65 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#64 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#62 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#63 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#83] +Results [2]: [c_customer_id#71 AS customer_id#84, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#65 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#64 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#62 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#63 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#83 AS year_total#85] (55) Filter [codegen id : 24] -Input [2]: [customer_id#85, year_total#86] -Condition : (isnotnull(year_total#86) AND (year_total#86 > 0.000000)) +Input [2]: [customer_id#84, year_total#85] +Condition : (isnotnull(year_total#85) AND (year_total#85 > 0.000000)) (56) Exchange -Input [2]: [customer_id#85, year_total#86] -Arguments: hashpartitioning(customer_id#85, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [customer_id#84, year_total#85] +Arguments: hashpartitioning(customer_id#84, 5), ENSURE_REQUIREMENTS, [id=#86] (57) Sort [codegen id : 25] -Input [2]: [customer_id#85, year_total#86] -Arguments: [customer_id#85 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#84, year_total#85] +Arguments: [customer_id#84 ASC NULLS FIRST], false, 0 (58) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#85] +Right keys [1]: [customer_id#84] Join condition: None (59) Project [codegen id : 26] -Output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#86] -Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#85, year_total#86] +Output [11]: [customer_id#26, year_total#27, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#85] +Input [12]: [customer_id#26, year_total#27, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, customer_id#84, year_total#85] (60) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, cs_sold_date_sk#93] +Output [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#93), dynamicpruningexpression(cs_sold_date_sk#93 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#92), dynamicpruningexpression(cs_sold_date_sk#92 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, cs_sold_date_sk#93] +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] (62) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, cs_sold_date_sk#93] -Condition : isnotnull(cs_bill_customer_sk#88) +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92] +Condition : isnotnull(cs_bill_customer_sk#87) (63) ReusedExchange [Reuses operator id: 126] -Output [2]: [d_date_sk#94, d_year#95] +Output [2]: [d_date_sk#93, d_year#94] (64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#93] -Right keys [1]: [d_date_sk#94] +Left keys [1]: [cs_sold_date_sk#92] +Right keys [1]: [d_date_sk#93] Join condition: None (65) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, d_year#95] -Input [8]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, cs_sold_date_sk#93, d_date_sk#94, d_year#95] +Output [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Input [8]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, cs_sold_date_sk#92, d_date_sk#93, d_year#94] (66) Exchange -Input [6]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, d_year#95] -Arguments: hashpartitioning(cs_bill_customer_sk#88, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Arguments: hashpartitioning(cs_bill_customer_sk#87, 5), ENSURE_REQUIREMENTS, [id=#95] (67) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, d_year#95] -Arguments: [cs_bill_customer_sk#88 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (68) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#97, c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104] +Output [8]: [c_customer_sk#96, c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103] (69) Sort [codegen id : 31] -Input [8]: [c_customer_sk#97, c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104] -Arguments: [c_customer_sk#97 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#96, c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103] +Arguments: [c_customer_sk#96 ASC NULLS FIRST], false, 0 (70) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#88] -Right keys [1]: [c_customer_sk#97] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#96] Join condition: None (71) Project [codegen id : 32] -Output [12]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, d_year#95] -Input [14]: [cs_bill_customer_sk#88, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, d_year#95, c_customer_sk#97, c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104] +Output [12]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Input [14]: [cs_bill_customer_sk#87, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94, c_customer_sk#96, c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103] (72) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, cs_ext_discount_amt#89, cs_ext_sales_price#90, cs_ext_wholesale_cost#91, cs_ext_list_price#92, d_year#95] -Keys [8]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, d_year#95] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#92 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#91 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#89 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#90 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#105, isEmpty#106] -Results [10]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, d_year#95, sum#107, isEmpty#108] +Input [12]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, cs_ext_discount_amt#88, cs_ext_sales_price#89, cs_ext_wholesale_cost#90, cs_ext_list_price#91, d_year#94] +Keys [8]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, d_year#94] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#91 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#90 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#88 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#89 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#104, isEmpty#105] +Results [10]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, d_year#94, sum#106, isEmpty#107] (73) Exchange -Input [10]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, d_year#95, sum#107, isEmpty#108] -Arguments: hashpartitioning(c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, d_year#95, 5), ENSURE_REQUIREMENTS, [id=#109] +Input [10]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, d_year#94, sum#106, isEmpty#107] +Arguments: hashpartitioning(c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, d_year#94, 5), ENSURE_REQUIREMENTS, [id=#108] (74) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, d_year#95, sum#107, isEmpty#108] -Keys [8]: [c_customer_id#98, c_first_name#99, c_last_name#100, c_preferred_cust_flag#101, c_birth_country#102, c_login#103, c_email_address#104, d_year#95] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#92 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#91 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#89 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#90 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#92 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#91 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#89 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#90 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#110] -Results [2]: [c_customer_id#98 AS customer_id#111, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#92 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#91 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#89 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#90 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#110 AS year_total#112] +Input [10]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, d_year#94, sum#106, isEmpty#107] +Keys [8]: [c_customer_id#97, c_first_name#98, c_last_name#99, c_preferred_cust_flag#100, c_birth_country#101, c_login#102, c_email_address#103, d_year#94] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#91 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#90 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#88 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#89 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#91 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#90 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#88 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#89 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#83] +Results [2]: [c_customer_id#97 AS customer_id#109, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#91 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#90 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#88 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#89 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#83 AS year_total#110] (75) Exchange -Input [2]: [customer_id#111, year_total#112] -Arguments: hashpartitioning(customer_id#111, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [2]: [customer_id#109, year_total#110] +Arguments: hashpartitioning(customer_id#109, 5), ENSURE_REQUIREMENTS, [id=#111] (76) Sort [codegen id : 34] -Input [2]: [customer_id#111, year_total#112] -Arguments: [customer_id#111 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#109, year_total#110] +Arguments: [customer_id#109 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#111] -Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#112) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#60) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#109] +Join condition: (CASE WHEN (year_total#85 > 0.000000) THEN CheckOverflow((promote_precision(year_total#110) / promote_precision(year_total#85)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#59) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) (78) Project [codegen id : 35] -Output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#86, year_total#112] -Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#86, customer_id#111, year_total#112] +Output [10]: [customer_id#26, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#85, year_total#110] +Input [13]: [customer_id#26, year_total#27, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#85, customer_id#109, year_total#110] (79) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, ws_sold_date_sk#119] +Output [6]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, ws_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#119), dynamicpruningexpression(ws_sold_date_sk#119 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#117), dynamicpruningexpression(ws_sold_date_sk#117 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, ws_sold_date_sk#119] +Input [6]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, ws_sold_date_sk#117] (81) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, ws_sold_date_sk#119] -Condition : isnotnull(ws_bill_customer_sk#114) +Input [6]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, ws_sold_date_sk#117] +Condition : isnotnull(ws_bill_customer_sk#112) (82) ReusedExchange [Reuses operator id: 122] -Output [2]: [d_date_sk#120, d_year#121] +Output [2]: [d_date_sk#118, d_year#119] (83) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#119] -Right keys [1]: [d_date_sk#120] +Left keys [1]: [ws_sold_date_sk#117] +Right keys [1]: [d_date_sk#118] Join condition: None (84) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, d_year#121] -Input [8]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, ws_sold_date_sk#119, d_date_sk#120, d_year#121] +Output [6]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, d_year#119] +Input [8]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, ws_sold_date_sk#117, d_date_sk#118, d_year#119] (85) Exchange -Input [6]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, d_year#121] -Arguments: hashpartitioning(ws_bill_customer_sk#114, 5), ENSURE_REQUIREMENTS, [id=#122] +Input [6]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, d_year#119] +Arguments: hashpartitioning(ws_bill_customer_sk#112, 5), ENSURE_REQUIREMENTS, [id=#120] (86) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, d_year#121] -Arguments: [ws_bill_customer_sk#114 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, d_year#119] +Arguments: [ws_bill_customer_sk#112 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] +Output [8]: [c_customer_sk#121, c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128] (88) Sort [codegen id : 40] -Input [8]: [c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] -Arguments: [c_customer_sk#123 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#121, c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128] +Arguments: [c_customer_sk#121 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#114] -Right keys [1]: [c_customer_sk#123] +Left keys [1]: [ws_bill_customer_sk#112] +Right keys [1]: [c_customer_sk#121] Join condition: None (90) Project [codegen id : 41] -Output [12]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, d_year#121] -Input [14]: [ws_bill_customer_sk#114, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, d_year#121, c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] +Output [12]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, d_year#119] +Input [14]: [ws_bill_customer_sk#112, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, d_year#119, c_customer_sk#121, c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128] (91) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, ws_ext_discount_amt#115, ws_ext_sales_price#116, ws_ext_wholesale_cost#117, ws_ext_list_price#118, d_year#121] -Keys [8]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, d_year#121] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#118 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#117 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#115 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#116 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [10]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, d_year#121, sum#133, isEmpty#134] +Input [12]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, ws_ext_discount_amt#113, ws_ext_sales_price#114, ws_ext_wholesale_cost#115, ws_ext_list_price#116, d_year#119] +Keys [8]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, d_year#119] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#116 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#115 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#113 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#114 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#129, isEmpty#130] +Results [10]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, d_year#119, sum#131, isEmpty#132] (92) Exchange -Input [10]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, d_year#121, sum#133, isEmpty#134] -Arguments: hashpartitioning(c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, d_year#121, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [10]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, d_year#119, sum#131, isEmpty#132] +Arguments: hashpartitioning(c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, d_year#119, 5), ENSURE_REQUIREMENTS, [id=#133] (93) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, d_year#121, sum#133, isEmpty#134] -Keys [8]: [c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130, d_year#121] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#118 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#117 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#115 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#116 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#118 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#117 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#115 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#116 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#136] -Results [2]: [c_customer_id#124 AS customer_id#137, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#118 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#117 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#115 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#116 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#136 AS year_total#138] +Input [10]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, d_year#119, sum#131, isEmpty#132] +Keys [8]: [c_customer_id#122, c_first_name#123, c_last_name#124, c_preferred_cust_flag#125, c_birth_country#126, c_login#127, c_email_address#128, d_year#119] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#116 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#115 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#113 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#114 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#116 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#115 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#113 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#114 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134] +Results [2]: [c_customer_id#122 AS customer_id#135, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#116 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#115 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#113 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#114 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134 AS year_total#136] (94) Filter [codegen id : 42] -Input [2]: [customer_id#137, year_total#138] -Condition : (isnotnull(year_total#138) AND (year_total#138 > 0.000000)) +Input [2]: [customer_id#135, year_total#136] +Condition : (isnotnull(year_total#136) AND (year_total#136 > 0.000000)) (95) Exchange -Input [2]: [customer_id#137, year_total#138] -Arguments: hashpartitioning(customer_id#137, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [2]: [customer_id#135, year_total#136] +Arguments: hashpartitioning(customer_id#135, 5), ENSURE_REQUIREMENTS, [id=#137] (96) Sort [codegen id : 43] -Input [2]: [customer_id#137, year_total#138] -Arguments: [customer_id#137 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#135, year_total#136] +Arguments: [customer_id#135 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#137] +Right keys [1]: [customer_id#135] Join condition: None (98) Project [codegen id : 44] -Output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#86, year_total#112, year_total#138] -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#86, year_total#112, customer_id#137, year_total#138] +Output [11]: [customer_id#26, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#85, year_total#110, year_total#136] +Input [12]: [customer_id#26, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#85, year_total#110, customer_id#135, year_total#136] (99) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, ws_sold_date_sk#145] +Output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#145), dynamicpruningexpression(ws_sold_date_sk#145 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, ws_sold_date_sk#145] +Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] (101) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, ws_sold_date_sk#145] -Condition : isnotnull(ws_bill_customer_sk#140) +Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] +Condition : isnotnull(ws_bill_customer_sk#138) (102) ReusedExchange [Reuses operator id: 126] -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#144, d_year#145] (103) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#145] -Right keys [1]: [d_date_sk#146] +Left keys [1]: [ws_sold_date_sk#143] +Right keys [1]: [d_date_sk#144] Join condition: None (104) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, d_year#147] -Input [8]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, ws_sold_date_sk#145, d_date_sk#146, d_year#147] +Output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#145] +Input [8]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143, d_date_sk#144, d_year#145] (105) Exchange -Input [6]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, d_year#147] -Arguments: hashpartitioning(ws_bill_customer_sk#140, 5), ENSURE_REQUIREMENTS, [id=#148] +Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#145] +Arguments: hashpartitioning(ws_bill_customer_sk#138, 5), ENSURE_REQUIREMENTS, [id=#146] (106) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, d_year#147] -Arguments: [ws_bill_customer_sk#140 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#145] +Arguments: [ws_bill_customer_sk#138 ASC NULLS FIRST], false, 0 (107) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#149, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156] +Output [8]: [c_customer_sk#147, c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154] (108) Sort [codegen id : 49] -Input [8]: [c_customer_sk#149, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156] -Arguments: [c_customer_sk#149 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#147, c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154] +Arguments: [c_customer_sk#147 ASC NULLS FIRST], false, 0 (109) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#140] -Right keys [1]: [c_customer_sk#149] +Left keys [1]: [ws_bill_customer_sk#138] +Right keys [1]: [c_customer_sk#147] Join condition: None (110) Project [codegen id : 50] -Output [12]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, d_year#147] -Input [14]: [ws_bill_customer_sk#140, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, d_year#147, c_customer_sk#149, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156] +Output [12]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#145] +Input [14]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#145, c_customer_sk#147, c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154] (111) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, ws_ext_discount_amt#141, ws_ext_sales_price#142, ws_ext_wholesale_cost#143, ws_ext_list_price#144, d_year#147] -Keys [8]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, d_year#147] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#144 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#143 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#141 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#142 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#157, isEmpty#158] -Results [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, d_year#147, sum#159, isEmpty#160] +Input [12]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#145] +Keys [8]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, d_year#145] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#142 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#141 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#139 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#140 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [10]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, d_year#145, sum#157, isEmpty#158] (112) Exchange -Input [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, d_year#147, sum#159, isEmpty#160] -Arguments: hashpartitioning(c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, d_year#147, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [10]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, d_year#145, sum#157, isEmpty#158] +Arguments: hashpartitioning(c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, d_year#145, 5), ENSURE_REQUIREMENTS, [id=#159] (113) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, d_year#147, sum#159, isEmpty#160] -Keys [8]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#154, c_login#155, c_email_address#156, d_year#147] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#144 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#143 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#141 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#142 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#144 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#143 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#141 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#142 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#162] -Results [2]: [c_customer_id#150 AS customer_id#163, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#144 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#143 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#141 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#142 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#162 AS year_total#164] +Input [10]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, d_year#145, sum#157, isEmpty#158] +Keys [8]: [c_customer_id#148, c_first_name#149, c_last_name#150, c_preferred_cust_flag#151, c_birth_country#152, c_login#153, c_email_address#154, d_year#145] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#142 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#141 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#139 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#140 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#142 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#141 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#139 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#140 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134] +Results [2]: [c_customer_id#148 AS customer_id#160, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#142 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#141 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#139 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#140 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134 AS year_total#161] (114) Exchange -Input [2]: [customer_id#163, year_total#164] -Arguments: hashpartitioning(customer_id#163, 5), ENSURE_REQUIREMENTS, [id=#165] +Input [2]: [customer_id#160, year_total#161] +Arguments: hashpartitioning(customer_id#160, 5), ENSURE_REQUIREMENTS, [id=#162] (115) Sort [codegen id : 52] -Input [2]: [customer_id#163, year_total#164] -Arguments: [customer_id#163 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#160, year_total#161] +Arguments: [customer_id#160 ASC NULLS FIRST], false, 0 (116) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#163] -Join condition: (CASE WHEN (year_total#86 > 0.000000) THEN CheckOverflow((promote_precision(year_total#112) / promote_precision(year_total#86)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#138 > 0.000000) THEN CheckOverflow((promote_precision(year_total#164) / promote_precision(year_total#138)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#160] +Join condition: (CASE WHEN (year_total#85 > 0.000000) THEN CheckOverflow((promote_precision(year_total#110) / promote_precision(year_total#85)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#136 > 0.000000) THEN CheckOverflow((promote_precision(year_total#161) / promote_precision(year_total#136)), DecimalType(38,14), true) ELSE null END) (117) Project [codegen id : 53] -Output [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#86, year_total#112, year_total#138, customer_id#163, year_total#164] +Output [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] +Input [13]: [customer_id#26, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#85, year_total#110, year_total#136, customer_id#160, year_total#161] (118) TakeOrderedAndProject -Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Arguments: 100, [customer_id#53 ASC NULLS FIRST, customer_first_name#54 ASC NULLS FIRST, customer_last_name#55 ASC NULLS FIRST, customer_preferred_cust_flag#56 ASC NULLS FIRST, customer_birth_country#57 ASC NULLS FIRST, customer_login#58 ASC NULLS FIRST, customer_email_address#59 ASC NULLS FIRST], [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] +Input [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] +Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] ===== Subqueries ===== @@ -678,7 +678,7 @@ Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2001)) AND isnotnull(d_date_sk (122) BroadcastExchange Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#163] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (126) @@ -703,14 +703,14 @@ Condition : ((isnotnull(d_year#37) AND (d_year#37 = 2002)) AND isnotnull(d_date_ (126) BroadcastExchange Input [2]: [d_date_sk#36, d_year#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#167] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#164] -Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 60 Hosting Expression = cs_sold_date_sk#93 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 60 Hosting Expression = cs_sold_date_sk#92 IN dynamicpruning#35 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#119 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#117 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#145 IN dynamicpruning#35 +Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#35 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index b95940f0c6292..f1817db3faef7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -254,369 +254,369 @@ Arguments: hashpartitioning(c_customer_id#28, c_first_name#29, c_last_name#30, c Input [10]: [c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34, d_year#44, sum#47, isEmpty#48] Keys [8]: [c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34, d_year#44] Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#38 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#36 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#37 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#38 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#36 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#37 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#50] -Results [8]: [c_customer_id#28 AS customer_id#51, c_first_name#29 AS customer_first_name#52, c_last_name#30 AS customer_last_name#53, c_preferred_cust_flag#31 AS customer_preferred_cust_flag#54, c_birth_country#32 AS customer_birth_country#55, c_login#33 AS customer_login#56, c_email_address#34 AS customer_email_address#57, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#38 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#36 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#37 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#50 AS year_total#58] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#38 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#36 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#37 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#24] +Results [8]: [c_customer_id#28 AS customer_id#50, c_first_name#29 AS customer_first_name#51, c_last_name#30 AS customer_last_name#52, c_preferred_cust_flag#31 AS customer_preferred_cust_flag#53, c_birth_country#32 AS customer_birth_country#54, c_login#33 AS customer_login#55, c_email_address#34 AS customer_email_address#56, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#38 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#36 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#37 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#24 AS year_total#57] (32) BroadcastExchange -Input [8]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#59] +Input [8]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (33) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#51] +Right keys [1]: [customer_id#50] Join condition: None (34) Scan parquet default.customer -Output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] +Output [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] +Input [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] (36) Filter [codegen id : 10] -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Condition : (isnotnull(c_customer_sk#60) AND isnotnull(c_customer_id#61)) +Input [8]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (37) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#68, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73] +Output [6]: [cs_bill_customer_sk#67, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#73), dynamicpruningexpression(cs_sold_date_sk#73 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#68, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73] +Input [6]: [cs_bill_customer_sk#67, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72] (39) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#68, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73] -Condition : isnotnull(cs_bill_customer_sk#68) +Input [6]: [cs_bill_customer_sk#67, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_bill_customer_sk#67) (40) BroadcastExchange -Input [6]: [cs_bill_customer_sk#68, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [6]: [cs_bill_customer_sk#67, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#60] -Right keys [1]: [cs_bill_customer_sk#68] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [cs_bill_customer_sk#67] Join condition: None (42) Project [codegen id : 10] -Output [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73] -Input [14]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, cs_bill_customer_sk#68, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73] +Output [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72] +Input [14]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, cs_bill_customer_sk#67, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72] (43) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#75, d_year#76] +Output [2]: [d_date_sk#74, d_year#75] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#73] -Right keys [1]: [d_date_sk#75] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#74] Join condition: None (45) Project [codegen id : 10] -Output [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, d_year#76] -Input [14]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, cs_sold_date_sk#73, d_date_sk#75, d_year#76] +Output [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, d_year#75] +Input [14]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, cs_sold_date_sk#72, d_date_sk#74, d_year#75] (46) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, cs_ext_discount_amt#69, cs_ext_sales_price#70, cs_ext_wholesale_cost#71, cs_ext_list_price#72, d_year#76] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#76] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#71 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#69 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#70 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#77, isEmpty#78] -Results [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#76, sum#79, isEmpty#80] +Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, cs_ext_discount_amt#68, cs_ext_sales_price#69, cs_ext_wholesale_cost#70, cs_ext_list_price#71, d_year#75] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#75] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#71 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#70 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#68 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#69 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#75, sum#78, isEmpty#79] (47) Exchange -Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#76, sum#79, isEmpty#80] -Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#76, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#75, sum#78, isEmpty#79] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#75, 5), ENSURE_REQUIREMENTS, [id=#80] (48) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#76, sum#79, isEmpty#80] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#76] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#71 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#69 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#70 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#71 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#69 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#70 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#82] -Results [2]: [c_customer_id#61 AS customer_id#83, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#71 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#69 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#70 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#82 AS year_total#84] +Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#75, sum#78, isEmpty#79] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#64, c_login#65, c_email_address#66, d_year#75] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#71 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#70 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#68 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#69 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#71 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#70 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#68 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#69 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81] +Results [2]: [c_customer_id#60 AS customer_id#82, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#71 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#70 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#68 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#69 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81 AS year_total#83] (49) Filter [codegen id : 11] -Input [2]: [customer_id#83, year_total#84] -Condition : (isnotnull(year_total#84) AND (year_total#84 > 0.000000)) +Input [2]: [customer_id#82, year_total#83] +Condition : (isnotnull(year_total#83) AND (year_total#83 > 0.000000)) (50) BroadcastExchange -Input [2]: [customer_id#83, year_total#84] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#85] +Input [2]: [customer_id#82, year_total#83] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#84] (51) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#83] +Right keys [1]: [customer_id#82] Join condition: None (52) Project [codegen id : 24] -Output [11]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, year_total#84] -Input [12]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, customer_id#83, year_total#84] +Output [11]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57, year_total#83] +Input [12]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57, customer_id#82, year_total#83] (53) Scan parquet default.customer -Output [8]: [c_customer_sk#86, c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93] +Output [8]: [c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (54) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#86, c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93] +Input [8]: [c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92] (55) Filter [codegen id : 14] -Input [8]: [c_customer_sk#86, c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93] -Condition : (isnotnull(c_customer_sk#86) AND isnotnull(c_customer_id#87)) +Input [8]: [c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92] +Condition : (isnotnull(c_customer_sk#85) AND isnotnull(c_customer_id#86)) (56) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99] +Output [6]: [cs_bill_customer_sk#93, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#99), dynamicpruningexpression(cs_sold_date_sk#99 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(cs_sold_date_sk#98), dynamicpruningexpression(cs_sold_date_sk#98 IN dynamicpruning#41)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99] +Input [6]: [cs_bill_customer_sk#93, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] (58) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99] -Condition : isnotnull(cs_bill_customer_sk#94) +Input [6]: [cs_bill_customer_sk#93, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] +Condition : isnotnull(cs_bill_customer_sk#93) (59) BroadcastExchange -Input [6]: [cs_bill_customer_sk#94, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#100] +Input [6]: [cs_bill_customer_sk#93, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#86] -Right keys [1]: [cs_bill_customer_sk#94] +Left keys [1]: [c_customer_sk#85] +Right keys [1]: [cs_bill_customer_sk#93] Join condition: None (61) Project [codegen id : 14] -Output [12]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99] -Input [14]: [c_customer_sk#86, c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, cs_bill_customer_sk#94, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99] +Output [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] +Input [14]: [c_customer_sk#85, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_bill_customer_sk#93, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98] (62) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#101, d_year#102] +Output [2]: [d_date_sk#100, d_year#101] (63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#99] -Right keys [1]: [d_date_sk#101] +Left keys [1]: [cs_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None (64) Project [codegen id : 14] -Output [12]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, d_year#102] -Input [14]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, cs_sold_date_sk#99, d_date_sk#101, d_year#102] +Output [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, d_year#101] +Input [14]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, cs_sold_date_sk#98, d_date_sk#100, d_year#101] (65) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, cs_ext_discount_amt#95, cs_ext_sales_price#96, cs_ext_wholesale_cost#97, cs_ext_list_price#98, d_year#102] -Keys [8]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, d_year#102] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#98 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#97 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#95 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#96 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#103, isEmpty#104] -Results [10]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, d_year#102, sum#105, isEmpty#106] +Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#94, cs_ext_sales_price#95, cs_ext_wholesale_cost#96, cs_ext_list_price#97, d_year#101] +Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#101] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#94 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#102, isEmpty#103] +Results [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#101, sum#104, isEmpty#105] (66) Exchange -Input [10]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, d_year#102, sum#105, isEmpty#106] -Arguments: hashpartitioning(c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, d_year#102, 5), ENSURE_REQUIREMENTS, [id=#107] +Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#101, sum#104, isEmpty#105] +Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#101, 5), ENSURE_REQUIREMENTS, [id=#106] (67) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, d_year#102, sum#105, isEmpty#106] -Keys [8]: [c_customer_id#87, c_first_name#88, c_last_name#89, c_preferred_cust_flag#90, c_birth_country#91, c_login#92, c_email_address#93, d_year#102] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#98 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#97 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#95 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#96 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#98 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#97 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#95 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#96 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#108] -Results [2]: [c_customer_id#87 AS customer_id#109, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#98 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#97 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#95 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#96 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#108 AS year_total#110] +Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#101, sum#104, isEmpty#105] +Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#101] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#94 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#94 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81] +Results [2]: [c_customer_id#86 AS customer_id#107, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#97 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#96 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#94 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#95 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#81 AS year_total#108] (68) BroadcastExchange -Input [2]: [customer_id#109, year_total#110] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#111] +Input [2]: [customer_id#107, year_total#108] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#109] (69) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#109] -Join condition: (CASE WHEN (year_total#84 > 0.000000) THEN CheckOverflow((promote_precision(year_total#110) / promote_precision(year_total#84)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#107] +Join condition: (CASE WHEN (year_total#83 > 0.000000) THEN CheckOverflow((promote_precision(year_total#108) / promote_precision(year_total#83)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#26 > 0.000000) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#26)), DecimalType(38,14), true) ELSE null END) (70) Project [codegen id : 24] -Output [10]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#84, year_total#110] -Input [13]: [customer_id#25, year_total#26, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#58, year_total#84, customer_id#109, year_total#110] +Output [10]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#83, year_total#108] +Input [13]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#57, year_total#83, customer_id#107, year_total#108] (71) Scan parquet default.customer -Output [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Output [8]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (72) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] +Input [8]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117] (73) Filter [codegen id : 18] -Input [8]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119] -Condition : (isnotnull(c_customer_sk#112) AND isnotnull(c_customer_id#113)) +Input [8]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117] +Condition : (isnotnull(c_customer_sk#110) AND isnotnull(c_customer_id#111)) (74) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Output [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#125), dynamicpruningexpression(ws_sold_date_sk#125 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Input [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] (76) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -Condition : isnotnull(ws_bill_customer_sk#120) +Input [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] +Condition : isnotnull(ws_bill_customer_sk#118) (77) BroadcastExchange -Input [6]: [ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#126] +Input [6]: [ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#124] (78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#112] -Right keys [1]: [ws_bill_customer_sk#120] +Left keys [1]: [c_customer_sk#110] +Right keys [1]: [ws_bill_customer_sk#118] Join condition: None (79) Project [codegen id : 18] -Output [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] -Input [14]: [c_customer_sk#112, c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_bill_customer_sk#120, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125] +Output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] +Input [14]: [c_customer_sk#110, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_bill_customer_sk#118, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123] (80) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#127, d_year#128] +Output [2]: [d_date_sk#125, d_year#126] (81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#125] -Right keys [1]: [d_date_sk#127] +Left keys [1]: [ws_sold_date_sk#123] +Right keys [1]: [d_date_sk#125] Join condition: None (82) Project [codegen id : 18] -Output [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] -Input [14]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, ws_sold_date_sk#125, d_date_sk#127, d_year#128] +Output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#126] +Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, ws_sold_date_sk#123, d_date_sk#125, d_year#126] (83) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, ws_ext_discount_amt#121, ws_ext_sales_price#122, ws_ext_wholesale_cost#123, ws_ext_list_price#124, d_year#128] -Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#129, isEmpty#130] -Results [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#131, isEmpty#132] +Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#126] +Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#127, isEmpty#128] +Results [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, sum#129, isEmpty#130] (84) Exchange -Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#131, isEmpty#132] -Arguments: hashpartitioning(c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, 5), ENSURE_REQUIREMENTS, [id=#133] +Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, sum#129, isEmpty#130] +Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, 5), ENSURE_REQUIREMENTS, [id=#131] (85) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128, sum#131, isEmpty#132] -Keys [8]: [c_customer_id#113, c_first_name#114, c_last_name#115, c_preferred_cust_flag#116, c_birth_country#117, c_login#118, c_email_address#119, d_year#128] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134] -Results [2]: [c_customer_id#113 AS customer_id#135, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#124 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#123 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#121 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#122 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#134 AS year_total#136] +Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126, sum#129, isEmpty#130] +Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#115, c_login#116, c_email_address#117, d_year#126] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132] +Results [2]: [c_customer_id#111 AS customer_id#133, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#122 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#121 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#119 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#120 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132 AS year_total#134] (86) Filter [codegen id : 19] -Input [2]: [customer_id#135, year_total#136] -Condition : (isnotnull(year_total#136) AND (year_total#136 > 0.000000)) +Input [2]: [customer_id#133, year_total#134] +Condition : (isnotnull(year_total#134) AND (year_total#134 > 0.000000)) (87) BroadcastExchange -Input [2]: [customer_id#135, year_total#136] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#137] +Input [2]: [customer_id#133, year_total#134] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#135] (88) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#135] +Right keys [1]: [customer_id#133] Join condition: None (89) Project [codegen id : 24] -Output [11]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#84, year_total#110, year_total#136] -Input [12]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#84, year_total#110, customer_id#135, year_total#136] +Output [11]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#83, year_total#108, year_total#134] +Input [12]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#83, year_total#108, customer_id#133, year_total#134] (90) Scan parquet default.customer -Output [8]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145] +Output [8]: [c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (91) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145] +Input [8]: [c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143] (92) Filter [codegen id : 22] -Input [8]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145] -Condition : (isnotnull(c_customer_sk#138) AND isnotnull(c_customer_id#139)) +Input [8]: [c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143] +Condition : (isnotnull(c_customer_sk#136) AND isnotnull(c_customer_id#137)) (93) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Output [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(ws_sold_date_sk#149), dynamicpruningexpression(ws_sold_date_sk#149 IN dynamicpruning#41)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (94) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] (95) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Condition : isnotnull(ws_bill_customer_sk#146) +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Condition : isnotnull(ws_bill_customer_sk#144) (96) BroadcastExchange -Input [6]: [ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#152] +Input [6]: [ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#150] (97) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#138] -Right keys [1]: [ws_bill_customer_sk#146] +Left keys [1]: [c_customer_sk#136] +Right keys [1]: [ws_bill_customer_sk#144] Join condition: None (98) Project [codegen id : 22] -Output [12]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] -Input [14]: [c_customer_sk#138, c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_bill_customer_sk#146, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151] +Output [12]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] +Input [14]: [c_customer_sk#136, c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_bill_customer_sk#144, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149] (99) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#153, d_year#154] +Output [2]: [d_date_sk#151, d_year#152] (100) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#151] -Right keys [1]: [d_date_sk#153] +Left keys [1]: [ws_sold_date_sk#149] +Right keys [1]: [d_date_sk#151] Join condition: None (101) Project [codegen id : 22] -Output [12]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#154] -Input [14]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, ws_sold_date_sk#151, d_date_sk#153, d_year#154] +Output [12]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#152] +Input [14]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, ws_sold_date_sk#149, d_date_sk#151, d_year#152] (102) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#154] -Keys [8]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#155, isEmpty#156] -Results [10]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, sum#157, isEmpty#158] +Input [12]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, ws_ext_discount_amt#145, ws_ext_sales_price#146, ws_ext_wholesale_cost#147, ws_ext_list_price#148, d_year#152] +Keys [8]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#152] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#153, isEmpty#154] +Results [10]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#152, sum#155, isEmpty#156] (103) Exchange -Input [10]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, sum#157, isEmpty#158] -Arguments: hashpartitioning(c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [10]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#152, sum#155, isEmpty#156] +Arguments: hashpartitioning(c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#152, 5), ENSURE_REQUIREMENTS, [id=#157] (104) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154, sum#157, isEmpty#158] -Keys [8]: [c_customer_id#139, c_first_name#140, c_last_name#141, c_preferred_cust_flag#142, c_birth_country#143, c_login#144, c_email_address#145, d_year#154] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#160] -Results [2]: [c_customer_id#139 AS customer_id#161, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#150 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#149 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#147 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#148 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#160 AS year_total#162] +Input [10]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#152, sum#155, isEmpty#156] +Keys [8]: [c_customer_id#137, c_first_name#138, c_last_name#139, c_preferred_cust_flag#140, c_birth_country#141, c_login#142, c_email_address#143, d_year#152] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132] +Results [2]: [c_customer_id#137 AS customer_id#158, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#148 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#147 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#145 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#146 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#132 AS year_total#159] (105) BroadcastExchange -Input [2]: [customer_id#161, year_total#162] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#163] +Input [2]: [customer_id#158, year_total#159] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#160] (106) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#161] -Join condition: (CASE WHEN (year_total#84 > 0.000000) THEN CheckOverflow((promote_precision(year_total#110) / promote_precision(year_total#84)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#136 > 0.000000) THEN CheckOverflow((promote_precision(year_total#162) / promote_precision(year_total#136)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#158] +Join condition: (CASE WHEN (year_total#83 > 0.000000) THEN CheckOverflow((promote_precision(year_total#108) / promote_precision(year_total#83)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#134 > 0.000000) THEN CheckOverflow((promote_precision(year_total#159) / promote_precision(year_total#134)), DecimalType(38,14), true) ELSE null END) (107) Project [codegen id : 24] -Output [7]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] -Input [13]: [customer_id#25, customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57, year_total#84, year_total#110, year_total#136, customer_id#161, year_total#162] +Output [7]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] +Input [13]: [customer_id#25, customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56, year_total#83, year_total#108, year_total#134, customer_id#158, year_total#159] (108) TakeOrderedAndProject -Input [7]: [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] -Arguments: 100, [customer_id#51 ASC NULLS FIRST, customer_first_name#52 ASC NULLS FIRST, customer_last_name#53 ASC NULLS FIRST, customer_preferred_cust_flag#54 ASC NULLS FIRST, customer_birth_country#55 ASC NULLS FIRST, customer_login#56 ASC NULLS FIRST, customer_email_address#57 ASC NULLS FIRST], [customer_id#51, customer_first_name#52, customer_last_name#53, customer_preferred_cust_flag#54, customer_birth_country#55, customer_login#56, customer_email_address#57] +Input [7]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] +Arguments: 100, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULLS FIRST, customer_last_name#52 ASC NULLS FIRST, customer_preferred_cust_flag#53 ASC NULLS FIRST, customer_birth_country#54 ASC NULLS FIRST, customer_login#55 ASC NULLS FIRST, customer_email_address#56 ASC NULLS FIRST], [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] ===== Subqueries ===== @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ (112) BroadcastExchange Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#164] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#161] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#40 IN dynamicpruning#41 BroadcastExchange (116) @@ -668,14 +668,14 @@ Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_ (116) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#165] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#162] -Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#73 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#99 IN dynamicpruning#41 +Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#98 IN dynamicpruning#41 -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#125 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#41 +Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#149 IN dynamicpruning#41 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index 686036a835e25..529b9c8282db5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -207,77 +207,77 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] Keys [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33] Functions [1]: [sum(UnscaledValue(ss_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#36] -Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#36,17,2) AS sum_sales#37] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#21] +Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#21,17,2) AS sum_sales#22] (36) Exchange -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#36] (37) Sort [codegen id : 21] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (38) Window -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#22 AS sum_sales#38, rn#37] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22, rn#37] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1)] Join condition: None (43) Project [codegen id : 24] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] (45) Sort [codegen id : 33] -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] +Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#22 AS sum_sales#47, rn#46] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22, rn#46] (48) Exchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] +Arguments: hashpartitioning(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] (49) Sort [codegen id : 35] -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] +Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#38 AS psum#49, sum_sales#47 AS nsum#50] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] (52) TakeOrderedAndProject -Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 1617f96c6c4d1..4f69eb1367b8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -180,69 +180,69 @@ Output [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_ye Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum#32] Keys [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31] Functions [1]: [sum(UnscaledValue(ss_sales_price#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#33))#34] -Results [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(ss_sales_price#33))#34,17,2) AS sum_sales#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#33))#20] +Results [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(ss_sales_price#33))#20,17,2) AS sum_sales#21] (31) Exchange -Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: hashpartitioning(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: hashpartitioning(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, 5), ENSURE_REQUIREMENTS, [id=#34] (32) Sort [codegen id : 13] -Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35] +Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21] Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST, s_company_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 (33) Window -Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] -Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#21 AS sum_sales#36, rn#35] +Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35] (35) BroadcastExchange -Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#38] +Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#36, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#37] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#37 + 1)] +Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#35 + 1)] Join condition: None (37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36] +Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#36, rn#35] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21] (39) Sort [codegen id : 20] -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, s_store_name#40, s_company_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#21 AS sum_sales#45, rn#44] +Input [8]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21, rn#44] (42) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#47] +Input [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#46] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] +Right keys [5]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, (rn#44 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#47, sum_sales#45 AS nsum#48] +Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] (45) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] +Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index 0f22f27fed99b..ef8d64cee2c4a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -207,77 +207,77 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] Keys [5]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31] Functions [1]: [sum(UnscaledValue(cs_sales_price#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#34] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#34,17,2) AS sum_sales#35] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#20] +Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#20,17,2) AS sum_sales#21] (36) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#34] (37) Sort [codegen id : 21] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 (38) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#21 AS sum_sales#36, rn#35] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#35 + 1), 5), ENSURE_REQUIREMENTS, [id=#37] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#35 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#35 + 1)] Join condition: None (43) Project [codegen id : 24] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] (45) Sort [codegen id : 33] -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] +Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#21 AS sum_sales#44, rn#43] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21, rn#43] (48) Exchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] +Arguments: hashpartitioning(i_category#38, i_brand#39, cc_name#40, (rn#43 - 1), 5), ENSURE_REQUIREMENTS, [id=#45] (49) Sort [codegen id : 35] -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, (rn#43 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] +Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#43 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#46, sum_sales#44 AS nsum#47] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] (52) TakeOrderedAndProject -Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 83ad4872f96c4..a3b9279528ba9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -180,69 +180,69 @@ Output [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum#30] Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum#30] Keys [5]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29] Functions [1]: [sum(UnscaledValue(cs_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#31))#32] -Results [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, MakeDecimal(sum(UnscaledValue(cs_sales_price#31))#32,17,2) AS sum_sales#33] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#31))#19] +Results [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, MakeDecimal(sum(UnscaledValue(cs_sales_price#31))#19,17,2) AS sum_sales#20] (31) Exchange -Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33] -Arguments: hashpartitioning(i_category#25, i_brand#26, cc_name#27, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20] +Arguments: hashpartitioning(i_category#25, i_brand#26, cc_name#27, 5), ENSURE_REQUIREMENTS, [id=#32] (32) Sort [codegen id : 13] -Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33] +Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20] Arguments: [i_category#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, cc_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST], false, 0 (33) Window -Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33] -Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] +Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20] +Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] -Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33, rn#35] +Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#20 AS sum_sales#34, rn#33] +Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20, rn#33] (35) BroadcastExchange -Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#36] +Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#34, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#35] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#35 + 1)] +Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#33 + 1)] Join condition: None (37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#34] +Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#34, rn#33] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Output [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20] (39) Sort [codegen id : 20] -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20] +Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20] +Arguments: [rank(d_year#39, d_moy#40) windowspecdefinition(i_category#36, i_brand#37, cc_name#38, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#36, i_brand#37, cc_name#38], [d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] +Output [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#20 AS sum_sales#42, rn#41] +Input [7]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20, rn#41] (42) BroadcastExchange -Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#44] +Input [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#43] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] +Right keys [4]: [i_category#36, i_brand#37, cc_name#38, (rn#41 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#34 AS psum#44, sum_sales#42 AS nsum#45] +Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#34, i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] (45) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] +Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt index cf22f178dd33f..7f2f33b2c1799 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt @@ -164,87 +164,87 @@ Output [10]: [s_store_name#39 AS s_store_name1#44, d_week_seq#5 AS d_week_seq1#4 Input [11]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, s_store_name#39, d_week_seq#42] (26) ReusedExchange [Reuses operator id: 11] -Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] +Output [9]: [d_week_seq#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (27) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] -Keys [2]: [d_week_seq#54, ss_store_sk#55] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#63 = Sunday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Monday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Tuesday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Wednesday) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Thursday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Friday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Saturday ) THEN ss_sales_price#64 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#63 = Sunday ) THEN ss_sales_price#64 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Monday ) THEN ss_sales_price#64 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Tuesday ) THEN ss_sales_price#64 ELSE null END))#67, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Wednesday) THEN ss_sales_price#64 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Thursday ) THEN ss_sales_price#64 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Friday ) THEN ss_sales_price#64 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Saturday ) THEN ss_sales_price#64 ELSE null END))#71] -Results [9]: [d_week_seq#54, ss_store_sk#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Sunday ) THEN ss_sales_price#64 ELSE null END))#65,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Monday ) THEN ss_sales_price#64 ELSE null END))#66,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Tuesday ) THEN ss_sales_price#64 ELSE null END))#67,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Wednesday) THEN ss_sales_price#64 ELSE null END))#68,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Thursday ) THEN ss_sales_price#64 ELSE null END))#69,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Friday ) THEN ss_sales_price#64 ELSE null END))#70,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Saturday ) THEN ss_sales_price#64 ELSE null END))#71,17,2) AS sat_sales#36] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#25,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29,17,2) AS sat_sales#36] (28) Scan parquet default.store -Output [2]: [s_store_sk#72, s_store_id#73] +Output [2]: [s_store_sk#61, s_store_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (29) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#72, s_store_id#73] +Input [2]: [s_store_sk#61, s_store_id#62] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#72, s_store_id#73] -Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) +Input [2]: [s_store_sk#61, s_store_id#62] +Condition : (isnotnull(s_store_sk#61) AND isnotnull(s_store_id#62)) (31) BroadcastExchange -Input [2]: [s_store_sk#72, s_store_id#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [2]: [s_store_sk#61, s_store_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#55] -Right keys [1]: [s_store_sk#72] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#61] Join condition: None (33) Project [codegen id : 9] -Output [9]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73] -Input [11]: [d_week_seq#54, ss_store_sk#55, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#72, s_store_id#73] +Output [9]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#62] +Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#61, s_store_id#62] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#75, d_week_seq#76] +Output [2]: [d_month_seq#64, d_week_seq#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct (35) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#75, d_week_seq#76] +Input [2]: [d_month_seq#64, d_week_seq#65] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#75, d_week_seq#76] -Condition : (((isnotnull(d_month_seq#75) AND (d_month_seq#75 >= 1224)) AND (d_month_seq#75 <= 1235)) AND isnotnull(d_week_seq#76)) +Input [2]: [d_month_seq#64, d_week_seq#65] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1224)) AND (d_month_seq#64 <= 1235)) AND isnotnull(d_week_seq#65)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#76] -Input [2]: [d_month_seq#75, d_week_seq#76] +Output [1]: [d_week_seq#65] +Input [2]: [d_month_seq#64, d_week_seq#65] (38) BroadcastExchange -Input [1]: [d_week_seq#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Input [1]: [d_week_seq#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#54] -Right keys [1]: [d_week_seq#76] +Left keys [1]: [d_week_seq#5] +Right keys [1]: [d_week_seq#65] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#54 AS d_week_seq2#78, s_store_id#73 AS s_store_id2#79, sun_sales#30 AS sun_sales2#80, mon_sales#31 AS mon_sales2#81, tue_sales#32 AS tue_sales2#82, wed_sales#33 AS wed_sales2#83, thu_sales#34 AS thu_sales2#84, fri_sales#35 AS fri_sales2#85, sat_sales#36 AS sat_sales2#86] -Input [10]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73, d_week_seq#76] +Output [9]: [d_week_seq#5 AS d_week_seq2#67, s_store_id#62 AS s_store_id2#68, sun_sales#30 AS sun_sales2#69, mon_sales#31 AS mon_sales2#70, tue_sales#32 AS tue_sales2#71, wed_sales#33 AS wed_sales2#72, thu_sales#34 AS thu_sales2#73, fri_sales#35 AS fri_sales2#74, sat_sales#36 AS sat_sales2#75] +Input [10]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#62, d_week_seq#65] (41) BroadcastExchange -Input [9]: [d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#87] +Input [9]: [d_week_seq2#67, s_store_id2#68, sun_sales2#69, mon_sales2#70, tue_sales2#71, wed_sales2#72, thu_sales2#73, fri_sales2#74, sat_sales2#75] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#76] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 52)] +Right keys [2]: [s_store_id2#68, (d_week_seq2#67 - 52)] Join condition: None (43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#80)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#81)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#82)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#83)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#84)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#85)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#86)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#69)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#77, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#70)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#78, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#71)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#79, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#72)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#80, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#73)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#81, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#74)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#82, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#75)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#83] +Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#67, s_store_id2#68, sun_sales2#69, mon_sales2#70, tue_sales2#71, wed_sales2#72, thu_sales2#73, fri_sales2#74, sat_sales2#75] (44) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] -Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] +Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index cf22f178dd33f..7f2f33b2c1799 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -164,87 +164,87 @@ Output [10]: [s_store_name#39 AS s_store_name1#44, d_week_seq#5 AS d_week_seq1#4 Input [11]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#38, s_store_name#39, d_week_seq#42] (26) ReusedExchange [Reuses operator id: 11] -Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] +Output [9]: [d_week_seq#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (27) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] -Keys [2]: [d_week_seq#54, ss_store_sk#55] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#63 = Sunday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Monday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Tuesday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Wednesday) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Thursday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Friday ) THEN ss_sales_price#64 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#63 = Saturday ) THEN ss_sales_price#64 ELSE null END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#63 = Sunday ) THEN ss_sales_price#64 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Monday ) THEN ss_sales_price#64 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Tuesday ) THEN ss_sales_price#64 ELSE null END))#67, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Wednesday) THEN ss_sales_price#64 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Thursday ) THEN ss_sales_price#64 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Friday ) THEN ss_sales_price#64 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#63 = Saturday ) THEN ss_sales_price#64 ELSE null END))#71] -Results [9]: [d_week_seq#54, ss_store_sk#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Sunday ) THEN ss_sales_price#64 ELSE null END))#65,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Monday ) THEN ss_sales_price#64 ELSE null END))#66,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Tuesday ) THEN ss_sales_price#64 ELSE null END))#67,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Wednesday) THEN ss_sales_price#64 ELSE null END))#68,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Thursday ) THEN ss_sales_price#64 ELSE null END))#69,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Friday ) THEN ss_sales_price#64 ELSE null END))#70,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#63 = Saturday ) THEN ss_sales_price#64 ELSE null END))#71,17,2) AS sat_sales#36] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 ELSE null END))#23,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#24,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#25,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#26,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#27,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#28,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#29,17,2) AS sat_sales#36] (28) Scan parquet default.store -Output [2]: [s_store_sk#72, s_store_id#73] +Output [2]: [s_store_sk#61, s_store_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct (29) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#72, s_store_id#73] +Input [2]: [s_store_sk#61, s_store_id#62] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#72, s_store_id#73] -Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) +Input [2]: [s_store_sk#61, s_store_id#62] +Condition : (isnotnull(s_store_sk#61) AND isnotnull(s_store_id#62)) (31) BroadcastExchange -Input [2]: [s_store_sk#72, s_store_id#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [2]: [s_store_sk#61, s_store_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#55] -Right keys [1]: [s_store_sk#72] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#61] Join condition: None (33) Project [codegen id : 9] -Output [9]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73] -Input [11]: [d_week_seq#54, ss_store_sk#55, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#72, s_store_id#73] +Output [9]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#62] +Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_sk#61, s_store_id#62] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#75, d_week_seq#76] +Output [2]: [d_month_seq#64, d_week_seq#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct (35) ColumnarToRow [codegen id : 8] -Input [2]: [d_month_seq#75, d_week_seq#76] +Input [2]: [d_month_seq#64, d_week_seq#65] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#75, d_week_seq#76] -Condition : (((isnotnull(d_month_seq#75) AND (d_month_seq#75 >= 1224)) AND (d_month_seq#75 <= 1235)) AND isnotnull(d_week_seq#76)) +Input [2]: [d_month_seq#64, d_week_seq#65] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1224)) AND (d_month_seq#64 <= 1235)) AND isnotnull(d_week_seq#65)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#76] -Input [2]: [d_month_seq#75, d_week_seq#76] +Output [1]: [d_week_seq#65] +Input [2]: [d_month_seq#64, d_week_seq#65] (38) BroadcastExchange -Input [1]: [d_week_seq#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Input [1]: [d_week_seq#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#54] -Right keys [1]: [d_week_seq#76] +Left keys [1]: [d_week_seq#5] +Right keys [1]: [d_week_seq#65] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#54 AS d_week_seq2#78, s_store_id#73 AS s_store_id2#79, sun_sales#30 AS sun_sales2#80, mon_sales#31 AS mon_sales2#81, tue_sales#32 AS tue_sales2#82, wed_sales#33 AS wed_sales2#83, thu_sales#34 AS thu_sales2#84, fri_sales#35 AS fri_sales2#85, sat_sales#36 AS sat_sales2#86] -Input [10]: [d_week_seq#54, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#73, d_week_seq#76] +Output [9]: [d_week_seq#5 AS d_week_seq2#67, s_store_id#62 AS s_store_id2#68, sun_sales#30 AS sun_sales2#69, mon_sales#31 AS mon_sales2#70, tue_sales#32 AS tue_sales2#71, wed_sales#33 AS wed_sales2#72, thu_sales#34 AS thu_sales2#73, fri_sales#35 AS fri_sales2#74, sat_sales#36 AS sat_sales2#75] +Input [10]: [d_week_seq#5, sun_sales#30, mon_sales#31, tue_sales#32, wed_sales#33, thu_sales#34, fri_sales#35, sat_sales#36, s_store_id#62, d_week_seq#65] (41) BroadcastExchange -Input [9]: [d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#87] +Input [9]: [d_week_seq2#67, s_store_id2#68, sun_sales2#69, mon_sales2#70, tue_sales2#71, wed_sales2#72, thu_sales2#73, fri_sales2#74, sat_sales2#75] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#76] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 52)] +Right keys [2]: [s_store_id2#68, (d_week_seq2#67 - 52)] Join condition: None (43) Project [codegen id : 10] -Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#80)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#88, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#81)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#89, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#82)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#90, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#83)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#91, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#84)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#92, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#85)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#93, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#86)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#94] -Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#78, s_store_id2#79, sun_sales2#80, mon_sales2#81, tue_sales2#82, wed_sales2#83, thu_sales2#84, fri_sales2#85, sat_sales2#86] +Output [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, CheckOverflow((promote_precision(sun_sales1#47) / promote_precision(sun_sales2#69)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#77, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#70)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#78, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#71)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#79, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#72)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#80, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#73)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#81, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#74)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#82, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#75)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#83] +Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mon_sales1#48, tue_sales1#49, wed_sales1#50, thu_sales1#51, fri_sales1#52, sat_sales1#53, d_week_seq2#67, s_store_id2#68, sun_sales2#69, mon_sales2#70, tue_sales2#71, wed_sales2#72, thu_sales2#73, fri_sales2#74, sat_sales2#75] (44) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] -Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#88, (mon_sales1 / mon_sales2)#89, (tue_sales1 / tue_sales2)#90, (wed_sales1 / wed_sales2)#91, (thu_sales1 / thu_sales2)#92, (fri_sales1 / fri_sales2)#93, (sat_sales1 / sat_sales2)#94] +Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 84c2fd54d5f4b..83d227688cf61 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -213,108 +213,108 @@ Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#35, ctr_state#36, ctr_total_return#37] (35) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] +Output [4]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#42), dynamicpruningexpression(cr_returned_date_sk#42 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(cr_returned_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] +Input [4]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24] (37) Filter [codegen id : 13] -Input [4]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42] -Condition : isnotnull(cr_returning_addr_sk#40) +Input [4]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_returning_addr_sk#22) (38) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#26] (39) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#42] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#26] Join condition: None (40) Project [codegen id : 13] -Output [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] -Input [5]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, cr_returned_date_sk#42, d_date_sk#43] +Output [3]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23] +Input [5]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, cr_returned_date_sk#24, d_date_sk#26] (41) Exchange -Input [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] -Arguments: hashpartitioning(cr_returning_addr_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23] +Arguments: hashpartitioning(cr_returning_addr_sk#22, 5), ENSURE_REQUIREMENTS, [id=#39] (42) Sort [codegen id : 14] -Input [3]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41] -Arguments: [cr_returning_addr_sk#40 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23] +Arguments: [cr_returning_addr_sk#22 ASC NULLS FIRST], false, 0 (43) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#45, ca_state#46] +Output [2]: [ca_address_sk#28, ca_state#29] (44) Sort [codegen id : 16] -Input [2]: [ca_address_sk#45, ca_state#46] -Arguments: [ca_address_sk#45 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#28, ca_state#29] +Arguments: [ca_address_sk#28 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] -Left keys [1]: [cr_returning_addr_sk#40] -Right keys [1]: [ca_address_sk#45] +Left keys [1]: [cr_returning_addr_sk#22] +Right keys [1]: [ca_address_sk#28] Join condition: None (46) Project [codegen id : 17] -Output [3]: [cr_returning_customer_sk#39, cr_return_amt_inc_tax#41, ca_state#46] -Input [5]: [cr_returning_customer_sk#39, cr_returning_addr_sk#40, cr_return_amt_inc_tax#41, ca_address_sk#45, ca_state#46] +Output [3]: [cr_returning_customer_sk#21, cr_return_amt_inc_tax#23, ca_state#29] +Input [5]: [cr_returning_customer_sk#21, cr_returning_addr_sk#22, cr_return_amt_inc_tax#23, ca_address_sk#28, ca_state#29] (47) HashAggregate [codegen id : 17] -Input [3]: [cr_returning_customer_sk#39, cr_return_amt_inc_tax#41, ca_state#46] -Keys [2]: [cr_returning_customer_sk#39, ca_state#46] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#41))] -Aggregate Attributes [1]: [sum#47] -Results [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] +Input [3]: [cr_returning_customer_sk#21, cr_return_amt_inc_tax#23, ca_state#29] +Keys [2]: [cr_returning_customer_sk#21, ca_state#29] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#23))] +Aggregate Attributes [1]: [sum#40] +Results [3]: [cr_returning_customer_sk#21, ca_state#29, sum#41] (48) Exchange -Input [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] -Arguments: hashpartitioning(cr_returning_customer_sk#39, ca_state#46, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [3]: [cr_returning_customer_sk#21, ca_state#29, sum#41] +Arguments: hashpartitioning(cr_returning_customer_sk#21, ca_state#29, 5), ENSURE_REQUIREMENTS, [id=#42] (49) HashAggregate [codegen id : 18] -Input [3]: [cr_returning_customer_sk#39, ca_state#46, sum#48] -Keys [2]: [cr_returning_customer_sk#39, ca_state#46] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#41))#50] -Results [2]: [ca_state#46 AS ctr_state#36, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#41))#50,17,2) AS ctr_total_return#37] +Input [3]: [cr_returning_customer_sk#21, ca_state#29, sum#41] +Keys [2]: [cr_returning_customer_sk#21, ca_state#29] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#23))#34] +Results [2]: [ca_state#29 AS ctr_state#36, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#23))#34,17,2) AS ctr_total_return#37] (50) HashAggregate [codegen id : 18] Input [2]: [ctr_state#36, ctr_total_return#37] Keys [1]: [ctr_state#36] Functions [1]: [partial_avg(ctr_total_return#37)] -Aggregate Attributes [2]: [sum#51, count#52] -Results [3]: [ctr_state#36, sum#53, count#54] +Aggregate Attributes [2]: [sum#43, count#44] +Results [3]: [ctr_state#36, sum#45, count#46] (51) Exchange -Input [3]: [ctr_state#36, sum#53, count#54] -Arguments: hashpartitioning(ctr_state#36, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [3]: [ctr_state#36, sum#45, count#46] +Arguments: hashpartitioning(ctr_state#36, 5), ENSURE_REQUIREMENTS, [id=#47] (52) HashAggregate [codegen id : 19] -Input [3]: [ctr_state#36, sum#53, count#54] +Input [3]: [ctr_state#36, sum#45, count#46] Keys [1]: [ctr_state#36] Functions [1]: [avg(ctr_total_return#37)] -Aggregate Attributes [1]: [avg(ctr_total_return#37)#56] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#37)#56) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#57, ctr_state#36 AS ctr_state#36#58] +Aggregate Attributes [1]: [avg(ctr_total_return#37)#48] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#37)#48) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#49, ctr_state#36 AS ctr_state#36#50] (53) Filter [codegen id : 19] -Input [2]: [(avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#57) +Input [2]: [(avg(ctr_total_return) * 1.2)#49, ctr_state#36#50] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#49) (54) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#59] +Input [2]: [(avg(ctr_total_return) * 1.2)#49, ctr_state#36#50] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#51] (55) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ctr_state#36] -Right keys [1]: [ctr_state#36#58] -Join condition: (cast(ctr_total_return#37 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#57) +Right keys [1]: [ctr_state#36#50] +Join condition: (cast(ctr_total_return#37 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#49) (56) Project [codegen id : 20] Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] -Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, (avg(ctr_total_return) * 1.2)#57, ctr_state#36#58] +Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#36, ctr_total_return#37, (avg(ctr_total_return) * 1.2)#49, ctr_state#36#50] (57) TakeOrderedAndProject Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] @@ -331,27 +331,27 @@ BroadcastExchange (62) (58) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_year#60] +Output [2]: [d_date_sk#26, d_year#52] 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 (59) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#26, d_year#60] +Input [2]: [d_date_sk#26, d_year#52] (60) Filter [codegen id : 1] -Input [2]: [d_date_sk#26, d_year#60] -Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2000)) AND isnotnull(d_date_sk#26)) +Input [2]: [d_date_sk#26, d_year#52] +Condition : ((isnotnull(d_year#52) AND (d_year#52 = 2000)) AND isnotnull(d_date_sk#26)) (61) Project [codegen id : 1] Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_year#60] +Input [2]: [d_date_sk#26, d_year#52] (62) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -Subquery:2 Hosting operator id = 35 Hosting Expression = cr_returned_date_sk#42 IN dynamicpruning#25 +Subquery:2 Hosting operator id = 35 Hosting Expression = cr_returned_date_sk#24 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 4a9a9cc153d0e..260224e41b7f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -126,154 +126,154 @@ Input [3]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16] Condition : isnotnull(ctr_total_return#16) (17) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] +Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#20), dynamicpruningexpression(cr_returned_date_sk#20 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] (19) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20] -Condition : isnotnull(cr_returning_addr_sk#18) +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Condition : isnotnull(cr_returning_addr_sk#2) (20) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#6] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (22) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19] -Input [5]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, cr_returned_date_sk#20, d_date_sk#21] +Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] (23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#7, ca_state#8] (24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#18] -Right keys [1]: [ca_address_sk#22] +Left keys [1]: [cr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#7] Join condition: None (25) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#17, cr_return_amt_inc_tax#19, ca_state#23] -Input [5]: [cr_returning_customer_sk#17, cr_returning_addr_sk#18, cr_return_amt_inc_tax#19, ca_address_sk#22, ca_state#23] +Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] (26) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#17, cr_return_amt_inc_tax#19, ca_state#23] -Keys [2]: [cr_returning_customer_sk#17, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#19))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] +Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum#17] +Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#18] (27) Exchange -Input [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] -Arguments: hashpartitioning(cr_returning_customer_sk#17, ca_state#23, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#18] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [id=#19] (28) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#17, ca_state#23, sum#25] -Keys [2]: [cr_returning_customer_sk#17, ca_state#23] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#19))#27] -Results [2]: [ca_state#23 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#19))#27,17,2) AS ctr_total_return#16] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#18] +Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#13] +Results [2]: [ca_state#8 AS ctr_state#15, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#13,17,2) AS ctr_total_return#16] (29) HashAggregate [codegen id : 7] Input [2]: [ctr_state#15, ctr_total_return#16] Keys [1]: [ctr_state#15] Functions [1]: [partial_avg(ctr_total_return#16)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ctr_state#15, sum#30, count#31] +Aggregate Attributes [2]: [sum#20, count#21] +Results [3]: [ctr_state#15, sum#22, count#23] (30) Exchange -Input [3]: [ctr_state#15, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [3]: [ctr_state#15, sum#22, count#23] +Arguments: hashpartitioning(ctr_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#15, sum#30, count#31] +Input [3]: [ctr_state#15, sum#22, count#23] Keys [1]: [ctr_state#15] Functions [1]: [avg(ctr_total_return#16)] -Aggregate Attributes [1]: [avg(ctr_total_return#16)#33] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#33) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#34, ctr_state#15 AS ctr_state#15#35] +Aggregate Attributes [1]: [avg(ctr_total_return#16)#25] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#16)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_state#15 AS ctr_state#15#27] (32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#34) +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_state#15#27] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) (33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#36] +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_state#15#27] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (34) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#15] -Right keys [1]: [ctr_state#15#35] -Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#34) +Right keys [1]: [ctr_state#15#27] +Join condition: (cast(ctr_total_return#16 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) (35) Project [codegen id : 11] Output [2]: [ctr_customer_sk#14, ctr_total_return#16] -Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#34, ctr_state#15#35] +Input [5]: [ctr_customer_sk#14, ctr_state#15, ctr_total_return#16, (avg(ctr_total_return) * 1.2)#26, ctr_state#15#27] (36) Scan parquet default.customer -Output [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +Output [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] (38) Filter [codegen id : 9] -Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_current_addr_sk#39)) +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) (39) BroadcastExchange -Input [6]: [c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#35] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#14] -Right keys [1]: [c_customer_sk#37] +Right keys [1]: [c_customer_sk#29] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] -Input [8]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#37, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42] +Output [6]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Input [8]: [ctr_customer_sk#14, ctr_total_return#16, c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] (42) Scan parquet default.customer_address -Output [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +Output [12]: [ca_address_sk#36, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +Input [12]: [ca_address_sk#36, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] (44) Filter [codegen id : 10] -Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] -Condition : ((isnotnull(ca_state#51) AND (ca_state#51 = GA)) AND isnotnull(ca_address_sk#44)) +Input [12]: [ca_address_sk#36, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] +Condition : ((isnotnull(ca_state#43) AND (ca_state#43 = GA)) AND isnotnull(ca_address_sk#36)) (45) BroadcastExchange -Input [12]: [ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] +Input [12]: [ca_address_sk#36, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] (46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#39] -Right keys [1]: [ca_address_sk#44] +Left keys [1]: [c_current_addr_sk#31] +Right keys [1]: [ca_address_sk#36] Join condition: None (47) Project [codegen id : 11] -Output [16]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] -Input [18]: [ctr_total_return#16, c_customer_id#38, c_current_addr_sk#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55] +Output [16]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16] +Input [18]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, ca_address_sk#36, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47] (48) TakeOrderedAndProject -Input [16]: [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, ca_street_number#45 ASC NULLS FIRST, ca_street_name#46 ASC NULLS FIRST, ca_street_type#47 ASC NULLS FIRST, ca_suite_number#48 ASC NULLS FIRST, ca_city#49 ASC NULLS FIRST, ca_county#50 ASC NULLS FIRST, ca_state#51 ASC NULLS FIRST, ca_zip#52 ASC NULLS FIRST, ca_country#53 ASC NULLS FIRST, ca_gmt_offset#54 ASC NULLS FIRST, ca_location_type#55 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#38, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#45, ca_street_name#46, ca_street_type#47, ca_suite_number#48, ca_city#49, ca_county#50, ca_state#51, ca_zip#52, ca_country#53, ca_gmt_offset#54, ca_location_type#55, ctr_total_return#16] +Input [16]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16] +Arguments: 100, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, ca_street_number#37 ASC NULLS FIRST, ca_street_name#38 ASC NULLS FIRST, ca_street_type#39 ASC NULLS FIRST, ca_suite_number#40 ASC NULLS FIRST, ca_city#41 ASC NULLS FIRST, ca_county#42 ASC NULLS FIRST, ca_state#43 ASC NULLS FIRST, ca_zip#44 ASC NULLS FIRST, ca_country#45 ASC NULLS FIRST, ca_gmt_offset#46 ASC NULLS FIRST, ca_location_type#47 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16] ===== Subqueries ===== @@ -286,27 +286,27 @@ BroadcastExchange (53) (49) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#57] +Output [2]: [d_date_sk#6, d_year#49] 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 (50) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#57] +Input [2]: [d_date_sk#6, d_year#49] (51) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_year#57] -Condition : ((isnotnull(d_year#57) AND (d_year#57 = 2000)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_year#49] +Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2000)) AND isnotnull(d_date_sk#6)) (52) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_year#57] +Input [2]: [d_date_sk#6, d_year#49] (53) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] -Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt index 6f41c9e2b09c5..1a24233541a26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.sf100/explain.txt @@ -158,32 +158,32 @@ Input [1]: [wr_order_number#15] Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (24) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#18, ws_order_number#19] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] (25) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#18, ws_order_number#19] -Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] Left keys [1]: [wr_order_number#15] -Right keys [1]: [ws_order_number#19] +Right keys [1]: [ws_order_number#10] Join condition: None (27) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#20, ws_order_number#21] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (28) Sort [codegen id : 15] -Input [2]: [ws_warehouse_sk#20, ws_order_number#21] -Arguments: [ws_order_number#21 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#13, ws_order_number#14] +Arguments: [ws_order_number#14 ASC NULLS FIRST], false, 0 (29) SortMergeJoin [codegen id : 16] -Left keys [1]: [ws_order_number#19] -Right keys [1]: [ws_order_number#21] -Join condition: NOT (ws_warehouse_sk#18 = ws_warehouse_sk#20) +Left keys [1]: [ws_order_number#10] +Right keys [1]: [ws_order_number#14] +Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#13) (30) Project [codegen id : 16] Output [1]: [wr_order_number#15] -Input [5]: [wr_order_number#15, ws_warehouse_sk#18, ws_order_number#19, ws_warehouse_sk#20, ws_order_number#21] +Input [5]: [wr_order_number#15, ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (31) SortMergeJoin [codegen id : 20] Left keys [1]: [ws_order_number#4] @@ -191,127 +191,127 @@ Right keys [1]: [wr_order_number#15] Join condition: None (32) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 17] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#18, ca_state#19] (34) Filter [codegen id : 17] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : ((isnotnull(ca_state#23) AND (ca_state#23 = IL)) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : ((isnotnull(ca_state#19) AND (ca_state#19 = IL)) AND isnotnull(ca_address_sk#18)) (35) Project [codegen id : 17] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_state#23] +Output [1]: [ca_address_sk#18] +Input [2]: [ca_address_sk#18, ca_state#19] (36) BroadcastExchange -Input [1]: [ca_address_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (37) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#18] Join condition: None (38) Project [codegen id : 20] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#22] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#18] (39) Scan parquet default.web_site -Output [2]: [web_site_sk#25, web_company_name#26] +Output [2]: [web_site_sk#21, web_company_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 18] -Input [2]: [web_site_sk#25, web_company_name#26] +Input [2]: [web_site_sk#21, web_company_name#22] (41) Filter [codegen id : 18] -Input [2]: [web_site_sk#25, web_company_name#26] -Condition : ((isnotnull(web_company_name#26) AND (web_company_name#26 = pri )) AND isnotnull(web_site_sk#25)) +Input [2]: [web_site_sk#21, web_company_name#22] +Condition : ((isnotnull(web_company_name#22) AND (web_company_name#22 = pri )) AND isnotnull(web_site_sk#21)) (42) Project [codegen id : 18] -Output [1]: [web_site_sk#25] -Input [2]: [web_site_sk#25, web_company_name#26] +Output [1]: [web_site_sk#21] +Input [2]: [web_site_sk#21, web_company_name#22] (43) BroadcastExchange -Input [1]: [web_site_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +Input [1]: [web_site_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (44) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#25] +Right keys [1]: [web_site_sk#21] Join condition: None (45) Project [codegen id : 20] Output [4]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#25] +Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#21] (46) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_date#29] +Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [2]: [d_date_sk#28, d_date#29] +Input [2]: [d_date_sk#24, d_date#25] (48) Filter [codegen id : 19] -Input [2]: [d_date_sk#28, d_date#29] -Condition : (((isnotnull(d_date#29) AND (d_date#29 >= 1999-02-01)) AND (d_date#29 <= 1999-04-02)) AND isnotnull(d_date_sk#28)) +Input [2]: [d_date_sk#24, d_date#25] +Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 1999-02-01)) AND (d_date#25 <= 1999-04-02)) AND isnotnull(d_date_sk#24)) (49) Project [codegen id : 19] -Output [1]: [d_date_sk#28] -Input [2]: [d_date_sk#28, d_date#29] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (50) BroadcastExchange -Input [1]: [d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (51) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#24] Join condition: None (52) Project [codegen id : 20] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#28] +Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#24] (53) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] -Results [3]: [ws_order_number#4, sum#33, sum#34] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] +Results [3]: [ws_order_number#4, sum#29, sum#30] (54) HashAggregate [codegen id : 20] -Input [3]: [ws_order_number#4, sum#33, sum#34] +Input [3]: [ws_order_number#4, sum#29, sum#30] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] -Results [3]: [ws_order_number#4, sum#33, sum#34] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] +Results [3]: [ws_order_number#4, sum#29, sum#30] (55) HashAggregate [codegen id : 20] -Input [3]: [ws_order_number#4, sum#33, sum#34] +Input [3]: [ws_order_number#4, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32, count(ws_order_number#4)#35] -Results [3]: [sum#33, sum#34, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#31] +Results [3]: [sum#29, sum#30, count#32] (56) Exchange -Input [3]: [sum#33, sum#34, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#29, sum#30, count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (57) HashAggregate [codegen id : 21] -Input [3]: [sum#33, sum#34, count#36] +Input [3]: [sum#29, sum#30, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32, count(ws_order_number#4)#35] -Results [3]: [count(ws_order_number#4)#35 AS order count #38, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#31,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#32,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#31] +Results [3]: [count(ws_order_number#4)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#27,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#28,17,2) AS total net profit #36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt index e464faa98a0ad..79519bfde99f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt @@ -159,36 +159,36 @@ Input [1]: [wr_order_number#15] Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (24) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#18, ws_order_number#19] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] (25) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#18, ws_order_number#19] -Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 (26) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#20, ws_order_number#21] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (27) Sort [codegen id : 14] -Input [2]: [ws_warehouse_sk#20, ws_order_number#21] -Arguments: [ws_order_number#21 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#13, ws_order_number#14] +Arguments: [ws_order_number#14 ASC NULLS FIRST], false, 0 (28) SortMergeJoin [codegen id : 15] -Left keys [1]: [ws_order_number#19] -Right keys [1]: [ws_order_number#21] -Join condition: NOT (ws_warehouse_sk#18 = ws_warehouse_sk#20) +Left keys [1]: [ws_order_number#10] +Right keys [1]: [ws_order_number#14] +Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#13) (29) Project [codegen id : 15] -Output [1]: [ws_order_number#19] -Input [4]: [ws_warehouse_sk#18, ws_order_number#19, ws_warehouse_sk#20, ws_order_number#21] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (30) SortMergeJoin [codegen id : 16] Left keys [1]: [wr_order_number#15] -Right keys [1]: [ws_order_number#19] +Right keys [1]: [ws_order_number#10] Join condition: None (31) Project [codegen id : 16] Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, ws_order_number#19] +Input [2]: [wr_order_number#15, ws_order_number#10] (32) SortMergeJoin [codegen id : 20] Left keys [1]: [ws_order_number#4] @@ -196,127 +196,127 @@ Right keys [1]: [wr_order_number#15] Join condition: None (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#23] +Output [2]: [d_date_sk#18, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 17] -Input [2]: [d_date_sk#22, d_date#23] +Input [2]: [d_date_sk#18, d_date#19] (35) Filter [codegen id : 17] -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1999-02-01)) AND (d_date#23 <= 1999-04-02)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#18, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-01)) AND (d_date#19 <= 1999-04-02)) AND isnotnull(d_date_sk#18)) (36) Project [codegen id : 17] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#23] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (38) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#18] Join condition: None (39) Project [codegen id : 20] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#22] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#18] (40) Scan parquet default.customer_address -Output [2]: [ca_address_sk#25, ca_state#26] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 18] -Input [2]: [ca_address_sk#25, ca_state#26] +Input [2]: [ca_address_sk#21, ca_state#22] (42) Filter [codegen id : 18] -Input [2]: [ca_address_sk#25, ca_state#26] -Condition : ((isnotnull(ca_state#26) AND (ca_state#26 = IL)) AND isnotnull(ca_address_sk#25)) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : ((isnotnull(ca_state#22) AND (ca_state#22 = IL)) AND isnotnull(ca_address_sk#21)) (43) Project [codegen id : 18] -Output [1]: [ca_address_sk#25] -Input [2]: [ca_address_sk#25, ca_state#26] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (44) BroadcastExchange -Input [1]: [ca_address_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (45) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#25] +Right keys [1]: [ca_address_sk#21] Join condition: None (46) Project [codegen id : 20] Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#25] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] (47) Scan parquet default.web_site -Output [2]: [web_site_sk#28, web_company_name#29] +Output [2]: [web_site_sk#24, web_company_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#28, web_company_name#29] +Input [2]: [web_site_sk#24, web_company_name#25] (49) Filter [codegen id : 19] -Input [2]: [web_site_sk#28, web_company_name#29] -Condition : ((isnotnull(web_company_name#29) AND (web_company_name#29 = pri )) AND isnotnull(web_site_sk#28)) +Input [2]: [web_site_sk#24, web_company_name#25] +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri )) AND isnotnull(web_site_sk#24)) (50) Project [codegen id : 19] -Output [1]: [web_site_sk#28] -Input [2]: [web_site_sk#28, web_company_name#29] +Output [1]: [web_site_sk#24] +Input [2]: [web_site_sk#24, web_company_name#25] (51) BroadcastExchange -Input [1]: [web_site_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [1]: [web_site_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (52) BroadcastHashJoin [codegen id : 20] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#28] +Right keys [1]: [web_site_sk#24] Join condition: None (53) Project [codegen id : 20] Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#28] +Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#24] (54) HashAggregate [codegen id : 20] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] -Results [3]: [ws_order_number#4, sum#33, sum#34] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] +Results [3]: [ws_order_number#4, sum#29, sum#30] (55) HashAggregate [codegen id : 20] -Input [3]: [ws_order_number#4, sum#33, sum#34] +Input [3]: [ws_order_number#4, sum#29, sum#30] Keys [1]: [ws_order_number#4] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32] -Results [3]: [ws_order_number#4, sum#33, sum#34] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] +Results [3]: [ws_order_number#4, sum#29, sum#30] (56) HashAggregate [codegen id : 20] -Input [3]: [ws_order_number#4, sum#33, sum#34] +Input [3]: [ws_order_number#4, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32, count(ws_order_number#4)#35] -Results [3]: [sum#33, sum#34, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#31] +Results [3]: [sum#29, sum#30, count#32] (57) Exchange -Input [3]: [sum#33, sum#34, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#29, sum#30, count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (58) HashAggregate [codegen id : 21] -Input [3]: [sum#33, sum#34, count#36] +Input [3]: [sum#29, sum#30, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#31, sum(UnscaledValue(ws_net_profit#6))#32, count(ws_order_number#4)#35] -Results [3]: [count(ws_order_number#4)#35 AS order count #38, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#31,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#32,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#31] +Results [3]: [count(ws_order_number#4)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#27,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#28,17,2) AS total net profit #36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index 9d018105c88af..732f510b80d1b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -242,201 +242,201 @@ Arguments: hashpartitioning(c_customer_id#34, c_first_name#35, c_last_name#36, d Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#31, c_preferred_cust_flag#37, c_birth_country#38, c_login#39, c_email_address#40, sum#42] Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#31, c_preferred_cust_flag#37, c_birth_country#38, c_login#39, c_email_address#40] Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#44] -Results [5]: [c_customer_id#34 AS customer_id#45, c_first_name#35 AS customer_first_name#46, c_last_name#36 AS customer_last_name#47, c_email_address#40 AS customer_email_address#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#49] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#21] +Results [5]: [c_customer_id#34 AS customer_id#44, c_first_name#35 AS customer_first_name#45, c_last_name#36 AS customer_last_name#46, c_email_address#40 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#27 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(8,2)))), DecimalType(8,2), true)))#21,18,2) AS year_total#48] (37) Exchange -Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] -Arguments: hashpartitioning(customer_id#45, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] +Arguments: hashpartitioning(customer_id#44, 5), ENSURE_REQUIREMENTS, [id=#49] (38) Sort [codegen id : 16] -Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] -Arguments: [customer_id#45 ASC NULLS FIRST], false, 0 +Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] +Arguments: [customer_id#44 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Right keys [1]: [customer_id#44] Join condition: None (40) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Output [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] (42) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#51) +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_customer_sk#50) (43) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#55, d_year#56] +Output [2]: [d_date_sk#54, d_year#55] (44) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#54] -Right keys [1]: [d_date_sk#55] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#54] Join condition: None (45) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56] -Input [6]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56] +Output [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#55] +Input [6]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, ws_sold_date_sk#53, d_date_sk#54, d_year#55] (46) Exchange -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56] -Arguments: hashpartitioning(ws_bill_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#55] +Arguments: hashpartitioning(ws_bill_customer_sk#50, 5), ENSURE_REQUIREMENTS, [id=#56] (47) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56] -Arguments: [ws_bill_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#55] +Arguments: [ws_bill_customer_sk#50 ASC NULLS FIRST], false, 0 (48) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] +Output [8]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64] (49) Sort [codegen id : 22] -Input [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] -Arguments: [c_customer_sk#58 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64] +Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#51] -Right keys [1]: [c_customer_sk#58] +Left keys [1]: [ws_bill_customer_sk#50] +Right keys [1]: [c_customer_sk#57] Join condition: None (51) Project [codegen id : 23] -Output [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56] -Input [12]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56, c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] +Output [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#55] +Input [12]: [ws_bill_customer_sk#50, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#55, c_customer_sk#57, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64] (52) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56] -Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#66] -Results [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#56, sum#67] +Input [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, ws_ext_discount_amt#51, ws_ext_list_price#52, d_year#55] +Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, d_year#55] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#51 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#65] +Results [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, d_year#55, sum#66] (53) Exchange -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#56, sum#67] -Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#56, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, d_year#55, sum#66] +Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, d_year#55, 5), ENSURE_REQUIREMENTS, [id=#67] (54) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#56, sum#67] -Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#56] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2), true)))#69] -Results [2]: [c_customer_id#59 AS customer_id#70, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2), true)))#69,18,2) AS year_total#71] +Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, d_year#55, sum#66] +Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#62, c_login#63, c_email_address#64, d_year#55] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#51 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#51 as decimal(8,2)))), DecimalType(8,2), true)))#68] +Results [2]: [c_customer_id#58 AS customer_id#69, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#51 as decimal(8,2)))), DecimalType(8,2), true)))#68,18,2) AS year_total#70] (55) Filter [codegen id : 24] -Input [2]: [customer_id#70, year_total#71] -Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) +Input [2]: [customer_id#69, year_total#70] +Condition : (isnotnull(year_total#70) AND (year_total#70 > 0.00)) (56) Exchange -Input [2]: [customer_id#70, year_total#71] -Arguments: hashpartitioning(customer_id#70, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [2]: [customer_id#69, year_total#70] +Arguments: hashpartitioning(customer_id#69, 5), ENSURE_REQUIREMENTS, [id=#71] (57) Sort [codegen id : 25] -Input [2]: [customer_id#70, year_total#71] -Arguments: [customer_id#70 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#69, year_total#70] +Arguments: [customer_id#69 ASC NULLS FIRST], false, 0 (58) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#70] +Right keys [1]: [customer_id#69] Join condition: None (59) Project [codegen id : 26] -Output [8]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#71] -Input [9]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, customer_id#70, year_total#71] +Output [8]: [customer_id#22, year_total#23, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#70] +Input [9]: [customer_id#22, year_total#23, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#69, year_total#70] (60) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Output [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] (62) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_bill_customer_sk#73) +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_bill_customer_sk#72) (63) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#76, d_year#77] (64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#76] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (65) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Input [6]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, ws_sold_date_sk#76, d_date_sk#77, d_year#78] +Output [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Input [6]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, ws_sold_date_sk#75, d_date_sk#76, d_year#77] (66) Exchange -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Arguments: hashpartitioning(ws_bill_customer_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Arguments: hashpartitioning(ws_bill_customer_sk#72, 5), ENSURE_REQUIREMENTS, [id=#78] (67) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Arguments: [ws_bill_customer_sk#73 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Arguments: [ws_bill_customer_sk#72 ASC NULLS FIRST], false, 0 (68) ReusedExchange [Reuses operator id: 12] -Output [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Output [8]: [c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86] (69) Sort [codegen id : 31] -Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] -Arguments: [c_customer_sk#80 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86] +Arguments: [c_customer_sk#79 ASC NULLS FIRST], false, 0 (70) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#73] -Right keys [1]: [c_customer_sk#80] +Left keys [1]: [ws_bill_customer_sk#72] +Right keys [1]: [c_customer_sk#79] Join condition: None (71) Project [codegen id : 32] -Output [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Input [12]: [ws_bill_customer_sk#73, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78, c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] +Output [10]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Input [12]: [ws_bill_customer_sk#72, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77, c_customer_sk#79, c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86] (72) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, ws_ext_discount_amt#74, ws_ext_list_price#75, d_year#78] -Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] +Input [10]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, ws_ext_discount_amt#73, ws_ext_list_price#74, d_year#77] +Keys [8]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#77] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#73 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#87] +Results [9]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#77, sum#88] (73) Exchange -Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] -Arguments: hashpartitioning(c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [9]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#77, sum#88] +Arguments: hashpartitioning(c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#77, 5), ENSURE_REQUIREMENTS, [id=#89] (74) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78, sum#89] -Keys [8]: [c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87, d_year#78] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91] -Results [2]: [c_customer_id#81 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#75 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#74 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] +Input [9]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#77, sum#88] +Keys [8]: [c_customer_id#80, c_first_name#81, c_last_name#82, c_preferred_cust_flag#83, c_birth_country#84, c_login#85, c_email_address#86, d_year#77] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#73 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#73 as decimal(8,2)))), DecimalType(8,2), true)))#68] +Results [2]: [c_customer_id#80 AS customer_id#90, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#73 as decimal(8,2)))), DecimalType(8,2), true)))#68,18,2) AS year_total#91] (75) Exchange -Input [2]: [customer_id#92, year_total#93] -Arguments: hashpartitioning(customer_id#92, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [customer_id#90, year_total#91] +Arguments: hashpartitioning(customer_id#90, 5), ENSURE_REQUIREMENTS, [id=#92] (76) Sort [codegen id : 34] -Input [2]: [customer_id#92, year_total#93] -Arguments: [customer_id#92 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#90, year_total#91] +Arguments: [customer_id#90 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#92] -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN CheckOverflow((promote_precision(year_total#93) / promote_precision(year_total#71)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#90] +Join condition: (CASE WHEN (year_total#70 > 0.00) THEN CheckOverflow((promote_precision(year_total#91) / promote_precision(year_total#70)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#48) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) (78) Project [codegen id : 35] -Output [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] -Input [10]: [customer_id#22, year_total#23, customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49, year_total#71, customer_id#92, year_total#93] +Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Input [10]: [customer_id#22, year_total#23, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#70, customer_id#90, year_total#91] (79) TakeOrderedAndProject -Input [4]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] -Arguments: 100, [customer_id#45 ASC NULLS FIRST, customer_first_name#46 ASC NULLS FIRST, customer_last_name#47 ASC NULLS FIRST, customer_email_address#48 ASC NULLS FIRST], [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48] +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] ===== Subqueries ===== @@ -463,7 +463,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2001)) AND isnotnull(d_date_sk (83) BroadcastExchange Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (87) @@ -488,10 +488,10 @@ Condition : ((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND isnotnull(d_date_ (87) BroadcastExchange Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#29 +Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#29 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index b371e4a9c8135..cb7fe2568123f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -217,195 +217,195 @@ Arguments: hashpartitioning(c_customer_id#24, c_first_name#25, c_last_name#26, d Input [9]: [c_customer_id#24, c_first_name#25, c_last_name#26, d_year#38, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, sum#40] Keys [8]: [c_customer_id#24, c_first_name#25, c_last_name#26, d_year#38, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30] Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#42] -Results [5]: [c_customer_id#24 AS customer_id#43, c_first_name#25 AS customer_first_name#44, c_last_name#26 AS customer_last_name#45, c_email_address#30 AS customer_email_address#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#42,18,2) AS year_total#47] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#20] +Results [5]: [c_customer_id#24 AS customer_id#42, c_first_name#25 AS customer_first_name#43, c_last_name#26 AS customer_last_name#44, c_email_address#30 AS customer_email_address#45, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#33 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#32 as decimal(8,2)))), DecimalType(8,2), true)))#20,18,2) AS year_total#46] (32) BroadcastExchange -Input [5]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [5]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45, year_total#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] (33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#42] Join condition: None (34) Scan parquet default.customer -Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Output [8]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Input [8]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55] (36) Filter [codegen id : 10] -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Condition : (isnotnull(c_customer_sk#49) AND isnotnull(c_customer_id#50)) +Input [8]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55] +Condition : (isnotnull(c_customer_sk#48) AND isnotnull(c_customer_id#49)) (37) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] +Output [4]: [ws_bill_customer_sk#56, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#59), dynamicpruningexpression(ws_sold_date_sk#59 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] +Input [4]: [ws_bill_customer_sk#56, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59] (39) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_customer_sk#57) +Input [4]: [ws_bill_customer_sk#56, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59] +Condition : isnotnull(ws_bill_customer_sk#56) (40) BroadcastExchange -Input [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#61] +Input [4]: [ws_bill_customer_sk#56, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#49] -Right keys [1]: [ws_bill_customer_sk#57] +Left keys [1]: [c_customer_sk#48] +Right keys [1]: [ws_bill_customer_sk#56] Join condition: None (42) Project [codegen id : 10] -Output [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] -Input [12]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] +Output [10]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59] +Input [12]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, ws_bill_customer_sk#56, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59] (43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#61, d_year#62] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#60] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ws_sold_date_sk#59] +Right keys [1]: [d_date_sk#61] Join condition: None (45) Project [codegen id : 10] -Output [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, ws_ext_discount_amt#58, ws_ext_list_price#59, d_year#63] -Input [12]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] +Output [10]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, ws_ext_discount_amt#57, ws_ext_list_price#58, d_year#62] +Input [12]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, ws_ext_discount_amt#57, ws_ext_list_price#58, ws_sold_date_sk#59, d_date_sk#61, d_year#62] (46) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, ws_ext_discount_amt#58, ws_ext_list_price#59, d_year#63] -Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#59 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#58 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#64] -Results [9]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#63, sum#65] +Input [10]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, ws_ext_discount_amt#57, ws_ext_list_price#58, d_year#62] +Keys [8]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, d_year#62] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#57 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#63] +Results [9]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, d_year#62, sum#64] (47) Exchange -Input [9]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#63, sum#65] -Arguments: hashpartitioning(c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#63, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [9]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, d_year#62, sum#64] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, d_year#62, 5), ENSURE_REQUIREMENTS, [id=#65] (48) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#63, sum#65] -Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#63] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#59 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#58 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#59 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#58 as decimal(8,2)))), DecimalType(8,2), true)))#67] -Results [2]: [c_customer_id#50 AS customer_id#68, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#59 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#58 as decimal(8,2)))), DecimalType(8,2), true)))#67,18,2) AS year_total#69] +Input [9]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, d_year#62, sum#64] +Keys [8]: [c_customer_id#49, c_first_name#50, c_last_name#51, c_preferred_cust_flag#52, c_birth_country#53, c_login#54, c_email_address#55, d_year#62] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#57 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#57 as decimal(8,2)))), DecimalType(8,2), true)))#66] +Results [2]: [c_customer_id#49 AS customer_id#67, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#57 as decimal(8,2)))), DecimalType(8,2), true)))#66,18,2) AS year_total#68] (49) Filter [codegen id : 11] -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) +Input [2]: [customer_id#67, year_total#68] +Condition : (isnotnull(year_total#68) AND (year_total#68 > 0.00)) (50) BroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#70] +Input [2]: [customer_id#67, year_total#68] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#69] (51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#68] +Right keys [1]: [customer_id#67] Join condition: None (52) Project [codegen id : 16] -Output [8]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, year_total#69] -Input [9]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, customer_id#68, year_total#69] +Output [8]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45, year_total#46, year_total#68] +Input [9]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45, year_total#46, customer_id#67, year_total#68] (53) Scan parquet default.customer -Output [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (54) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] (55) Filter [codegen id : 14] -Input [8]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78] -Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_customer_id#72)) +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(c_customer_id#71)) (56) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#35)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] (58) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] -Condition : isnotnull(ws_bill_customer_sk#79) +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#78) (59) BroadcastExchange -Input [4]: [ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#83] +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#71] -Right keys [1]: [ws_bill_customer_sk#79] +Left keys [1]: [c_customer_sk#70] +Right keys [1]: [ws_bill_customer_sk#78] Join condition: None (61) Project [codegen id : 14] -Output [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] -Input [12]: [c_customer_sk#71, c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_bill_customer_sk#79, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82] +Output [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Input [12]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] (62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#84, d_year#85] +Output [2]: [d_date_sk#83, d_year#84] (63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#82] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#83] Join condition: None (64) Project [codegen id : 14] -Output [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, d_year#85] -Input [12]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, ws_sold_date_sk#82, d_date_sk#84, d_year#85] +Output [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] +Input [12]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] (65) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, ws_ext_discount_amt#80, ws_ext_list_price#81, d_year#85] -Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#86] -Results [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] +Input [10]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#84] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#79 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#85] +Results [9]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#84, sum#86] (66) Exchange -Input [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] -Arguments: hashpartitioning(c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [9]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#84, sum#86] +Arguments: hashpartitioning(c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#84, 5), ENSURE_REQUIREMENTS, [id=#87] (67) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85, sum#87] -Keys [8]: [c_customer_id#72, c_first_name#73, c_last_name#74, c_preferred_cust_flag#75, c_birth_country#76, c_login#77, c_email_address#78, d_year#85] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))#89] -Results [2]: [c_customer_id#72 AS customer_id#90, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#80 as decimal(8,2)))), DecimalType(8,2), true)))#89,18,2) AS year_total#91] +Input [9]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#84, sum#86] +Keys [8]: [c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77, d_year#84] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#79 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#79 as decimal(8,2)))), DecimalType(8,2), true)))#66] +Results [2]: [c_customer_id#71 AS customer_id#88, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#79 as decimal(8,2)))), DecimalType(8,2), true)))#66,18,2) AS year_total#89] (68) BroadcastExchange -Input [2]: [customer_id#90, year_total#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#92] +Input [2]: [customer_id#88, year_total#89] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] (69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#21] -Right keys [1]: [customer_id#90] -Join condition: (CASE WHEN (year_total#69 > 0.00) THEN CheckOverflow((promote_precision(year_total#91) / promote_precision(year_total#69)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#47) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#88] +Join condition: (CASE WHEN (year_total#68 > 0.00) THEN CheckOverflow((promote_precision(year_total#89) / promote_precision(year_total#68)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#22 > 0.00) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#22)), DecimalType(38,20), true) ELSE 0E-20 END) (70) Project [codegen id : 16] -Output [4]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] -Input [10]: [customer_id#21, year_total#22, customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46, year_total#47, year_total#69, customer_id#90, year_total#91] +Output [4]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45] +Input [10]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45, year_total#46, year_total#68, customer_id#88, year_total#89] (71) TakeOrderedAndProject -Input [4]: [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] -Arguments: 100, [customer_id#43 ASC NULLS FIRST, customer_first_name#44 ASC NULLS FIRST, customer_last_name#45 ASC NULLS FIRST, customer_email_address#46 ASC NULLS FIRST], [customer_id#43, customer_first_name#44, customer_last_name#45, customer_email_address#46] +Input [4]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45] +Arguments: 100, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45] ===== Subqueries ===== @@ -432,7 +432,7 @@ Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_ (75) BroadcastExchange Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#91] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#34 IN dynamicpruning#35 BroadcastExchange (79) @@ -457,10 +457,10 @@ Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_ (79) BroadcastExchange Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#59 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#35 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#35 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 c0da9617cc440..5595e1a12b3fc 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 @@ -699,116 +699,116 @@ Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] (122) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#123, i_class_id#124, i_category_id#125, sum#126, isEmpty#127, sum#128] +Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] (123) HashAggregate [codegen id : 280] -Input [7]: [channel#63, i_brand_id#123, i_class_id#124, i_category_id#125, sum#126, isEmpty#127, sum#128] -Keys [4]: [channel#63, i_brand_id#123, i_class_id#124, i_category_id#125] +Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] +Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#129, sum(number_sales#65)#130] -Results [5]: [channel#63, i_brand_id#123, i_class_id#124, sum(sales#64)#129 AS sum_sales#121, sum(number_sales#65)#130 AS number_sales#122] +Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] +Results [5]: [channel#63, i_brand_id#49, i_class_id#50, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] (124) HashAggregate [codegen id : 280] -Input [5]: [channel#63, i_brand_id#123, i_class_id#124, sum_sales#121, number_sales#122] -Keys [3]: [channel#63, i_brand_id#123, i_class_id#124] +Input [5]: [channel#63, i_brand_id#49, i_class_id#50, sum_sales#121, number_sales#122] +Keys [3]: [channel#63, i_brand_id#49, i_class_id#50] Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] -Results [6]: [channel#63, i_brand_id#123, i_class_id#124, sum#134, isEmpty#135, sum#136] +Aggregate Attributes [3]: [sum#123, isEmpty#124, sum#125] +Results [6]: [channel#63, i_brand_id#49, i_class_id#50, sum#126, isEmpty#127, sum#128] (125) Exchange -Input [6]: [channel#63, i_brand_id#123, i_class_id#124, sum#134, isEmpty#135, sum#136] -Arguments: hashpartitioning(channel#63, i_brand_id#123, i_class_id#124, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [6]: [channel#63, i_brand_id#49, i_class_id#50, sum#126, isEmpty#127, sum#128] +Arguments: hashpartitioning(channel#63, i_brand_id#49, i_class_id#50, 5), ENSURE_REQUIREMENTS, [id=#129] (126) HashAggregate [codegen id : 281] -Input [6]: [channel#63, i_brand_id#123, i_class_id#124, sum#134, isEmpty#135, sum#136] -Keys [3]: [channel#63, i_brand_id#123, i_class_id#124] +Input [6]: [channel#63, i_brand_id#49, i_class_id#50, sum#126, isEmpty#127, sum#128] +Keys [3]: [channel#63, i_brand_id#49, i_class_id#50] Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#138, sum(number_sales#122)#139] -Results [6]: [channel#63, i_brand_id#123, i_class_id#124, null AS i_category_id#140, sum(sum_sales#121)#138 AS sum(sum_sales)#141, sum(number_sales#122)#139 AS sum(number_sales)#142] +Aggregate Attributes [2]: [sum(sum_sales#121)#130, sum(number_sales#122)#131] +Results [6]: [channel#63, i_brand_id#49, i_class_id#50, null AS i_category_id#132, sum(sum_sales#121)#130 AS sum(sum_sales)#133, sum(number_sales#122)#131 AS sum(number_sales)#134] (127) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#143, i_class_id#144, i_category_id#145, sum#146, isEmpty#147, sum#148] +Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] (128) HashAggregate [codegen id : 421] -Input [7]: [channel#63, i_brand_id#143, i_class_id#144, i_category_id#145, sum#146, isEmpty#147, sum#148] -Keys [4]: [channel#63, i_brand_id#143, i_class_id#144, i_category_id#145] +Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] +Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#149, sum(number_sales#65)#150] -Results [4]: [channel#63, i_brand_id#143, sum(sales#64)#149 AS sum_sales#121, sum(number_sales#65)#150 AS number_sales#122] +Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] +Results [4]: [channel#63, i_brand_id#49, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] (129) HashAggregate [codegen id : 421] -Input [4]: [channel#63, i_brand_id#143, sum_sales#121, number_sales#122] -Keys [2]: [channel#63, i_brand_id#143] +Input [4]: [channel#63, i_brand_id#49, sum_sales#121, number_sales#122] +Keys [2]: [channel#63, i_brand_id#49] Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#151, isEmpty#152, sum#153] -Results [5]: [channel#63, i_brand_id#143, sum#154, isEmpty#155, sum#156] +Aggregate Attributes [3]: [sum#135, isEmpty#136, sum#137] +Results [5]: [channel#63, i_brand_id#49, sum#138, isEmpty#139, sum#140] (130) Exchange -Input [5]: [channel#63, i_brand_id#143, sum#154, isEmpty#155, sum#156] -Arguments: hashpartitioning(channel#63, i_brand_id#143, 5), ENSURE_REQUIREMENTS, [id=#157] +Input [5]: [channel#63, i_brand_id#49, sum#138, isEmpty#139, sum#140] +Arguments: hashpartitioning(channel#63, i_brand_id#49, 5), ENSURE_REQUIREMENTS, [id=#141] (131) HashAggregate [codegen id : 422] -Input [5]: [channel#63, i_brand_id#143, sum#154, isEmpty#155, sum#156] -Keys [2]: [channel#63, i_brand_id#143] +Input [5]: [channel#63, i_brand_id#49, sum#138, isEmpty#139, sum#140] +Keys [2]: [channel#63, i_brand_id#49] Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#158, sum(number_sales#122)#159] -Results [6]: [channel#63, i_brand_id#143, null AS i_class_id#160, null AS i_category_id#161, sum(sum_sales#121)#158 AS sum(sum_sales)#162, sum(number_sales#122)#159 AS sum(number_sales)#163] +Aggregate Attributes [2]: [sum(sum_sales#121)#142, sum(number_sales#122)#143] +Results [6]: [channel#63, i_brand_id#49, null AS i_class_id#144, null AS i_category_id#145, sum(sum_sales#121)#142 AS sum(sum_sales)#146, sum(number_sales#122)#143 AS sum(number_sales)#147] (132) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#164, i_class_id#165, i_category_id#166, sum#167, isEmpty#168, sum#169] +Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] (133) HashAggregate [codegen id : 562] -Input [7]: [channel#63, i_brand_id#164, i_class_id#165, i_category_id#166, sum#167, isEmpty#168, sum#169] -Keys [4]: [channel#63, i_brand_id#164, i_class_id#165, i_category_id#166] +Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] +Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#170, sum(number_sales#65)#171] -Results [3]: [channel#63, sum(sales#64)#170 AS sum_sales#121, sum(number_sales#65)#171 AS number_sales#122] +Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] +Results [3]: [channel#63, sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] (134) HashAggregate [codegen id : 562] Input [3]: [channel#63, sum_sales#121, number_sales#122] Keys [1]: [channel#63] Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#172, isEmpty#173, sum#174] -Results [4]: [channel#63, sum#175, isEmpty#176, sum#177] +Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] +Results [4]: [channel#63, sum#151, isEmpty#152, sum#153] (135) Exchange -Input [4]: [channel#63, sum#175, isEmpty#176, sum#177] -Arguments: hashpartitioning(channel#63, 5), ENSURE_REQUIREMENTS, [id=#178] +Input [4]: [channel#63, sum#151, isEmpty#152, sum#153] +Arguments: hashpartitioning(channel#63, 5), ENSURE_REQUIREMENTS, [id=#154] (136) HashAggregate [codegen id : 563] -Input [4]: [channel#63, sum#175, isEmpty#176, sum#177] +Input [4]: [channel#63, sum#151, isEmpty#152, sum#153] Keys [1]: [channel#63] Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#179, sum(number_sales#122)#180] -Results [6]: [channel#63, null AS i_brand_id#181, null AS i_class_id#182, null AS i_category_id#183, sum(sum_sales#121)#179 AS sum(sum_sales)#184, sum(number_sales#122)#180 AS sum(number_sales)#185] +Aggregate Attributes [2]: [sum(sum_sales#121)#155, sum(number_sales#122)#156] +Results [6]: [channel#63, null AS i_brand_id#157, null AS i_class_id#158, null AS i_category_id#159, sum(sum_sales#121)#155 AS sum(sum_sales)#160, sum(number_sales#122)#156 AS sum(number_sales)#161] (137) ReusedExchange [Reuses operator id: 120] -Output [7]: [channel#63, i_brand_id#186, i_class_id#187, i_category_id#188, sum#189, isEmpty#190, sum#191] +Output [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] (138) HashAggregate [codegen id : 703] -Input [7]: [channel#63, i_brand_id#186, i_class_id#187, i_category_id#188, sum#189, isEmpty#190, sum#191] -Keys [4]: [channel#63, i_brand_id#186, i_class_id#187, i_category_id#188] +Input [7]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum#115, isEmpty#116, sum#117] +Keys [4]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51] Functions [2]: [sum(sales#64), sum(number_sales#65)] -Aggregate Attributes [2]: [sum(sales#64)#192, sum(number_sales#65)#193] -Results [2]: [sum(sales#64)#192 AS sum_sales#121, sum(number_sales#65)#193 AS number_sales#122] +Aggregate Attributes [2]: [sum(sales#64)#119, sum(number_sales#65)#120] +Results [2]: [sum(sales#64)#119 AS sum_sales#121, sum(number_sales#65)#120 AS number_sales#122] (139) HashAggregate [codegen id : 703] Input [2]: [sum_sales#121, number_sales#122] Keys: [] Functions [2]: [partial_sum(sum_sales#121), partial_sum(number_sales#122)] -Aggregate Attributes [3]: [sum#194, isEmpty#195, sum#196] -Results [3]: [sum#197, isEmpty#198, sum#199] +Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] +Results [3]: [sum#165, isEmpty#166, sum#167] (140) Exchange -Input [3]: [sum#197, isEmpty#198, sum#199] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#200] +Input [3]: [sum#165, isEmpty#166, sum#167] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#168] (141) HashAggregate [codegen id : 704] -Input [3]: [sum#197, isEmpty#198, sum#199] +Input [3]: [sum#165, isEmpty#166, sum#167] Keys: [] Functions [2]: [sum(sum_sales#121), sum(number_sales#122)] -Aggregate Attributes [2]: [sum(sum_sales#121)#201, sum(number_sales#122)#202] -Results [6]: [null AS channel#203, null AS i_brand_id#204, null AS i_class_id#205, null AS i_category_id#206, sum(sum_sales#121)#201 AS sum(sum_sales)#207, sum(number_sales#122)#202 AS sum(number_sales)#208] +Aggregate Attributes [2]: [sum(sum_sales#121)#169, sum(number_sales#122)#170] +Results [6]: [null AS channel#171, null AS i_brand_id#172, null AS i_class_id#173, null AS i_category_id#174, sum(sum_sales#121)#169 AS sum(sum_sales)#175, sum(number_sales#122)#170 AS sum(number_sales)#176] (142) Union @@ -821,7 +821,7 @@ Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sa (144) Exchange Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] -Arguments: hashpartitioning(channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122, 5), ENSURE_REQUIREMENTS, [id=#209] +Arguments: hashpartitioning(channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122, 5), ENSURE_REQUIREMENTS, [id=#177] (145) HashAggregate [codegen id : 706] Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] @@ -859,94 +859,94 @@ Subquery:1 Hosting operator id = 81 Hosting Expression = Subquery scalar-subquer (147) Scan parquet default.store_sales -Output [3]: [ss_quantity#210, ss_list_price#211, ss_sold_date_sk#212] +Output [3]: [ss_quantity#178, ss_list_price#179, ss_sold_date_sk#180] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#212), dynamicpruningexpression(ss_sold_date_sk#212 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#180), dynamicpruningexpression(ss_sold_date_sk#180 IN dynamicpruning#13)] ReadSchema: struct (148) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#210, ss_list_price#211, ss_sold_date_sk#212] +Input [3]: [ss_quantity#178, ss_list_price#179, ss_sold_date_sk#180] (149) ReusedExchange [Reuses operator id: 180] -Output [1]: [d_date_sk#213] +Output [1]: [d_date_sk#181] (150) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#212] -Right keys [1]: [d_date_sk#213] +Left keys [1]: [ss_sold_date_sk#180] +Right keys [1]: [d_date_sk#181] Join condition: None (151) Project [codegen id : 2] -Output [2]: [ss_quantity#210 AS quantity#214, ss_list_price#211 AS list_price#215] -Input [4]: [ss_quantity#210, ss_list_price#211, ss_sold_date_sk#212, d_date_sk#213] +Output [2]: [ss_quantity#178 AS quantity#182, ss_list_price#179 AS list_price#183] +Input [4]: [ss_quantity#178, ss_list_price#179, ss_sold_date_sk#180, d_date_sk#181] (152) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#216, cs_list_price#217, cs_sold_date_sk#218] +Output [3]: [cs_quantity#184, cs_list_price#185, cs_sold_date_sk#186] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#218), dynamicpruningexpression(cs_sold_date_sk#218 IN dynamicpruning#219)] +PartitionFilters: [isnotnull(cs_sold_date_sk#186), dynamicpruningexpression(cs_sold_date_sk#186 IN dynamicpruning#187)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#216, cs_list_price#217, cs_sold_date_sk#218] +Input [3]: [cs_quantity#184, cs_list_price#185, cs_sold_date_sk#186] (154) ReusedExchange [Reuses operator id: 170] -Output [1]: [d_date_sk#220] +Output [1]: [d_date_sk#188] (155) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#218] -Right keys [1]: [d_date_sk#220] +Left keys [1]: [cs_sold_date_sk#186] +Right keys [1]: [d_date_sk#188] Join condition: None (156) Project [codegen id : 4] -Output [2]: [cs_quantity#216 AS quantity#221, cs_list_price#217 AS list_price#222] -Input [4]: [cs_quantity#216, cs_list_price#217, cs_sold_date_sk#218, d_date_sk#220] +Output [2]: [cs_quantity#184 AS quantity#189, cs_list_price#185 AS list_price#190] +Input [4]: [cs_quantity#184, cs_list_price#185, cs_sold_date_sk#186, d_date_sk#188] (157) Scan parquet default.web_sales -Output [3]: [ws_quantity#223, ws_list_price#224, ws_sold_date_sk#225] +Output [3]: [ws_quantity#191, ws_list_price#192, ws_sold_date_sk#193] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#225), dynamicpruningexpression(ws_sold_date_sk#225 IN dynamicpruning#219)] +PartitionFilters: [isnotnull(ws_sold_date_sk#193), dynamicpruningexpression(ws_sold_date_sk#193 IN dynamicpruning#187)] ReadSchema: struct (158) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#223, ws_list_price#224, ws_sold_date_sk#225] +Input [3]: [ws_quantity#191, ws_list_price#192, ws_sold_date_sk#193] (159) ReusedExchange [Reuses operator id: 170] -Output [1]: [d_date_sk#226] +Output [1]: [d_date_sk#194] (160) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#225] -Right keys [1]: [d_date_sk#226] +Left keys [1]: [ws_sold_date_sk#193] +Right keys [1]: [d_date_sk#194] Join condition: None (161) Project [codegen id : 6] -Output [2]: [ws_quantity#223 AS quantity#227, ws_list_price#224 AS list_price#228] -Input [4]: [ws_quantity#223, ws_list_price#224, ws_sold_date_sk#225, d_date_sk#226] +Output [2]: [ws_quantity#191 AS quantity#195, ws_list_price#192 AS list_price#196] +Input [4]: [ws_quantity#191, ws_list_price#192, ws_sold_date_sk#193, d_date_sk#194] (162) Union (163) HashAggregate [codegen id : 7] -Input [2]: [quantity#214, list_price#215] +Input [2]: [quantity#182, list_price#183] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#214 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#215 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#229, count#230] -Results [2]: [sum#231, count#232] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#182 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#197, count#198] +Results [2]: [sum#199, count#200] (164) Exchange -Input [2]: [sum#231, count#232] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#233] +Input [2]: [sum#199, count#200] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#201] (165) HashAggregate [codegen id : 8] -Input [2]: [sum#231, count#232] +Input [2]: [sum#199, count#200] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#214 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#215 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#214 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#215 as decimal(12,2)))), DecimalType(18,2), true))#234] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#214 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#215 as decimal(12,2)))), DecimalType(18,2), true))#234 AS average_sales#235] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#182 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#182 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2), true))#202] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#182 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#183 as decimal(12,2)))), DecimalType(18,2), true))#202 AS average_sales#203] -Subquery:2 Hosting operator id = 147 Hosting Expression = ss_sold_date_sk#212 IN dynamicpruning#13 +Subquery:2 Hosting operator id = 147 Hosting Expression = ss_sold_date_sk#180 IN dynamicpruning#13 -Subquery:3 Hosting operator id = 152 Hosting Expression = cs_sold_date_sk#218 IN dynamicpruning#219 +Subquery:3 Hosting operator id = 152 Hosting Expression = cs_sold_date_sk#186 IN dynamicpruning#187 BroadcastExchange (170) +- * Project (169) +- * Filter (168) @@ -955,28 +955,28 @@ BroadcastExchange (170) (166) Scan parquet default.date_dim -Output [2]: [d_date_sk#220, d_year#236] +Output [2]: [d_date_sk#188, d_year#204] 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 (167) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#220, d_year#236] +Input [2]: [d_date_sk#188, d_year#204] (168) Filter [codegen id : 1] -Input [2]: [d_date_sk#220, d_year#236] -Condition : (((isnotnull(d_year#236) AND (d_year#236 >= 1998)) AND (d_year#236 <= 2000)) AND isnotnull(d_date_sk#220)) +Input [2]: [d_date_sk#188, d_year#204] +Condition : (((isnotnull(d_year#204) AND (d_year#204 >= 1998)) AND (d_year#204 <= 2000)) AND isnotnull(d_date_sk#188)) (169) Project [codegen id : 1] -Output [1]: [d_date_sk#220] -Input [2]: [d_date_sk#220, d_year#236] +Output [1]: [d_date_sk#188] +Input [2]: [d_date_sk#188, d_year#204] (170) BroadcastExchange -Input [1]: [d_date_sk#220] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#237] +Input [1]: [d_date_sk#188] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#205] -Subquery:4 Hosting operator id = 157 Hosting Expression = ws_sold_date_sk#225 IN dynamicpruning#219 +Subquery:4 Hosting operator id = 157 Hosting Expression = ws_sold_date_sk#193 IN dynamicpruning#187 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (175) @@ -987,26 +987,26 @@ BroadcastExchange (175) (171) Scan parquet default.date_dim -Output [3]: [d_date_sk#47, d_year#238, d_moy#239] +Output [3]: [d_date_sk#47, d_year#206, d_moy#207] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (172) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#47, d_year#238, d_moy#239] +Input [3]: [d_date_sk#47, d_year#206, d_moy#207] (173) Filter [codegen id : 1] -Input [3]: [d_date_sk#47, d_year#238, d_moy#239] -Condition : ((((isnotnull(d_year#238) AND isnotnull(d_moy#239)) AND (d_year#238 = 2000)) AND (d_moy#239 = 11)) AND isnotnull(d_date_sk#47)) +Input [3]: [d_date_sk#47, d_year#206, d_moy#207] +Condition : ((((isnotnull(d_year#206) AND isnotnull(d_moy#207)) AND (d_year#206 = 2000)) AND (d_moy#207 = 11)) AND isnotnull(d_date_sk#47)) (174) Project [codegen id : 1] Output [1]: [d_date_sk#47] -Input [3]: [d_date_sk#47, d_year#238, d_moy#239] +Input [3]: [d_date_sk#47, d_year#206, d_moy#207] (175) BroadcastExchange Input [1]: [d_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#240] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#208] Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) @@ -1017,26 +1017,26 @@ BroadcastExchange (180) (176) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#241] +Output [2]: [d_date_sk#14, d_year#209] 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 (177) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#241] +Input [2]: [d_date_sk#14, d_year#209] (178) Filter [codegen id : 1] -Input [2]: [d_date_sk#14, d_year#241] -Condition : (((isnotnull(d_year#241) AND (d_year#241 >= 1999)) AND (d_year#241 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#209] +Condition : (((isnotnull(d_year#209) AND (d_year#209 >= 1999)) AND (d_year#209 <= 2001)) AND isnotnull(d_date_sk#14)) (179) Project [codegen id : 1] Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#241] +Input [2]: [d_date_sk#14, d_year#209] (180) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#242] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#210] Subquery:7 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#13 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 c073e779d0c16..bd3290f8c55b4 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 @@ -604,116 +604,116 @@ Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] (103) ReusedExchange [Reuses operator id: 101] -Output [7]: [channel#58, i_brand_id#116, i_class_id#117, i_category_id#118, sum#119, isEmpty#120, sum#121] +Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] (104) HashAggregate [codegen id : 160] -Input [7]: [channel#58, i_brand_id#116, i_class_id#117, i_category_id#118, sum#119, isEmpty#120, sum#121] -Keys [4]: [channel#58, i_brand_id#116, i_class_id#117, i_category_id#118] +Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] +Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] -Aggregate Attributes [2]: [sum(sales#59)#122, sum(number_sales#60)#123] -Results [5]: [channel#58, i_brand_id#116, i_class_id#117, sum(sales#59)#122 AS sum_sales#114, sum(number_sales#60)#123 AS number_sales#115] +Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] +Results [5]: [channel#58, i_brand_id#44, i_class_id#45, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] (105) HashAggregate [codegen id : 160] -Input [5]: [channel#58, i_brand_id#116, i_class_id#117, sum_sales#114, number_sales#115] -Keys [3]: [channel#58, i_brand_id#116, i_class_id#117] +Input [5]: [channel#58, i_brand_id#44, i_class_id#45, sum_sales#114, number_sales#115] +Keys [3]: [channel#58, i_brand_id#44, i_class_id#45] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] -Results [6]: [channel#58, i_brand_id#116, i_class_id#117, sum#127, isEmpty#128, sum#129] +Aggregate Attributes [3]: [sum#116, isEmpty#117, sum#118] +Results [6]: [channel#58, i_brand_id#44, i_class_id#45, sum#119, isEmpty#120, sum#121] (106) Exchange -Input [6]: [channel#58, i_brand_id#116, i_class_id#117, sum#127, isEmpty#128, sum#129] -Arguments: hashpartitioning(channel#58, i_brand_id#116, i_class_id#117, 5), ENSURE_REQUIREMENTS, [id=#130] +Input [6]: [channel#58, i_brand_id#44, i_class_id#45, sum#119, isEmpty#120, sum#121] +Arguments: hashpartitioning(channel#58, i_brand_id#44, i_class_id#45, 5), ENSURE_REQUIREMENTS, [id=#122] (107) HashAggregate [codegen id : 161] -Input [6]: [channel#58, i_brand_id#116, i_class_id#117, sum#127, isEmpty#128, sum#129] -Keys [3]: [channel#58, i_brand_id#116, i_class_id#117] +Input [6]: [channel#58, i_brand_id#44, i_class_id#45, sum#119, isEmpty#120, sum#121] +Keys [3]: [channel#58, i_brand_id#44, i_class_id#45] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#131, sum(number_sales#115)#132] -Results [6]: [channel#58, i_brand_id#116, i_class_id#117, null AS i_category_id#133, sum(sum_sales#114)#131 AS sum(sum_sales)#134, sum(number_sales#115)#132 AS sum(number_sales)#135] +Aggregate Attributes [2]: [sum(sum_sales#114)#123, sum(number_sales#115)#124] +Results [6]: [channel#58, i_brand_id#44, i_class_id#45, null AS i_category_id#125, sum(sum_sales#114)#123 AS sum(sum_sales)#126, sum(number_sales#115)#124 AS sum(number_sales)#127] (108) ReusedExchange [Reuses operator id: 101] -Output [7]: [channel#58, i_brand_id#136, i_class_id#137, i_category_id#138, sum#139, isEmpty#140, sum#141] +Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] (109) HashAggregate [codegen id : 241] -Input [7]: [channel#58, i_brand_id#136, i_class_id#137, i_category_id#138, sum#139, isEmpty#140, sum#141] -Keys [4]: [channel#58, i_brand_id#136, i_class_id#137, i_category_id#138] +Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] +Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] -Aggregate Attributes [2]: [sum(sales#59)#142, sum(number_sales#60)#143] -Results [4]: [channel#58, i_brand_id#136, sum(sales#59)#142 AS sum_sales#114, sum(number_sales#60)#143 AS number_sales#115] +Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] +Results [4]: [channel#58, i_brand_id#44, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] (110) HashAggregate [codegen id : 241] -Input [4]: [channel#58, i_brand_id#136, sum_sales#114, number_sales#115] -Keys [2]: [channel#58, i_brand_id#136] +Input [4]: [channel#58, i_brand_id#44, sum_sales#114, number_sales#115] +Keys [2]: [channel#58, i_brand_id#44] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] -Results [5]: [channel#58, i_brand_id#136, sum#147, isEmpty#148, sum#149] +Aggregate Attributes [3]: [sum#128, isEmpty#129, sum#130] +Results [5]: [channel#58, i_brand_id#44, sum#131, isEmpty#132, sum#133] (111) Exchange -Input [5]: [channel#58, i_brand_id#136, sum#147, isEmpty#148, sum#149] -Arguments: hashpartitioning(channel#58, i_brand_id#136, 5), ENSURE_REQUIREMENTS, [id=#150] +Input [5]: [channel#58, i_brand_id#44, sum#131, isEmpty#132, sum#133] +Arguments: hashpartitioning(channel#58, i_brand_id#44, 5), ENSURE_REQUIREMENTS, [id=#134] (112) HashAggregate [codegen id : 242] -Input [5]: [channel#58, i_brand_id#136, sum#147, isEmpty#148, sum#149] -Keys [2]: [channel#58, i_brand_id#136] +Input [5]: [channel#58, i_brand_id#44, sum#131, isEmpty#132, sum#133] +Keys [2]: [channel#58, i_brand_id#44] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#151, sum(number_sales#115)#152] -Results [6]: [channel#58, i_brand_id#136, null AS i_class_id#153, null AS i_category_id#154, sum(sum_sales#114)#151 AS sum(sum_sales)#155, sum(number_sales#115)#152 AS sum(number_sales)#156] +Aggregate Attributes [2]: [sum(sum_sales#114)#135, sum(number_sales#115)#136] +Results [6]: [channel#58, i_brand_id#44, null AS i_class_id#137, null AS i_category_id#138, sum(sum_sales#114)#135 AS sum(sum_sales)#139, sum(number_sales#115)#136 AS sum(number_sales)#140] (113) ReusedExchange [Reuses operator id: 101] -Output [7]: [channel#58, i_brand_id#157, i_class_id#158, i_category_id#159, sum#160, isEmpty#161, sum#162] +Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] (114) HashAggregate [codegen id : 322] -Input [7]: [channel#58, i_brand_id#157, i_class_id#158, i_category_id#159, sum#160, isEmpty#161, sum#162] -Keys [4]: [channel#58, i_brand_id#157, i_class_id#158, i_category_id#159] +Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] +Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] -Aggregate Attributes [2]: [sum(sales#59)#163, sum(number_sales#60)#164] -Results [3]: [channel#58, sum(sales#59)#163 AS sum_sales#114, sum(number_sales#60)#164 AS number_sales#115] +Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] +Results [3]: [channel#58, sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] (115) HashAggregate [codegen id : 322] Input [3]: [channel#58, sum_sales#114, number_sales#115] Keys [1]: [channel#58] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#165, isEmpty#166, sum#167] -Results [4]: [channel#58, sum#168, isEmpty#169, sum#170] +Aggregate Attributes [3]: [sum#141, isEmpty#142, sum#143] +Results [4]: [channel#58, sum#144, isEmpty#145, sum#146] (116) Exchange -Input [4]: [channel#58, sum#168, isEmpty#169, sum#170] -Arguments: hashpartitioning(channel#58, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [4]: [channel#58, sum#144, isEmpty#145, sum#146] +Arguments: hashpartitioning(channel#58, 5), ENSURE_REQUIREMENTS, [id=#147] (117) HashAggregate [codegen id : 323] -Input [4]: [channel#58, sum#168, isEmpty#169, sum#170] +Input [4]: [channel#58, sum#144, isEmpty#145, sum#146] Keys [1]: [channel#58] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#172, sum(number_sales#115)#173] -Results [6]: [channel#58, null AS i_brand_id#174, null AS i_class_id#175, null AS i_category_id#176, sum(sum_sales#114)#172 AS sum(sum_sales)#177, sum(number_sales#115)#173 AS sum(number_sales)#178] +Aggregate Attributes [2]: [sum(sum_sales#114)#148, sum(number_sales#115)#149] +Results [6]: [channel#58, null AS i_brand_id#150, null AS i_class_id#151, null AS i_category_id#152, sum(sum_sales#114)#148 AS sum(sum_sales)#153, sum(number_sales#115)#149 AS sum(number_sales)#154] (118) ReusedExchange [Reuses operator id: 101] -Output [7]: [channel#58, i_brand_id#179, i_class_id#180, i_category_id#181, sum#182, isEmpty#183, sum#184] +Output [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] (119) HashAggregate [codegen id : 403] -Input [7]: [channel#58, i_brand_id#179, i_class_id#180, i_category_id#181, sum#182, isEmpty#183, sum#184] -Keys [4]: [channel#58, i_brand_id#179, i_class_id#180, i_category_id#181] +Input [7]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum#108, isEmpty#109, sum#110] +Keys [4]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46] Functions [2]: [sum(sales#59), sum(number_sales#60)] -Aggregate Attributes [2]: [sum(sales#59)#185, sum(number_sales#60)#186] -Results [2]: [sum(sales#59)#185 AS sum_sales#114, sum(number_sales#60)#186 AS number_sales#115] +Aggregate Attributes [2]: [sum(sales#59)#112, sum(number_sales#60)#113] +Results [2]: [sum(sales#59)#112 AS sum_sales#114, sum(number_sales#60)#113 AS number_sales#115] (120) HashAggregate [codegen id : 403] Input [2]: [sum_sales#114, number_sales#115] Keys: [] Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] -Aggregate Attributes [3]: [sum#187, isEmpty#188, sum#189] -Results [3]: [sum#190, isEmpty#191, sum#192] +Aggregate Attributes [3]: [sum#155, isEmpty#156, sum#157] +Results [3]: [sum#158, isEmpty#159, sum#160] (121) Exchange -Input [3]: [sum#190, isEmpty#191, sum#192] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#193] +Input [3]: [sum#158, isEmpty#159, sum#160] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] (122) HashAggregate [codegen id : 404] -Input [3]: [sum#190, isEmpty#191, sum#192] +Input [3]: [sum#158, isEmpty#159, sum#160] Keys: [] Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] -Aggregate Attributes [2]: [sum(sum_sales#114)#194, sum(number_sales#115)#195] -Results [6]: [null AS channel#196, null AS i_brand_id#197, null AS i_class_id#198, null AS i_category_id#199, sum(sum_sales#114)#194 AS sum(sum_sales)#200, sum(number_sales#115)#195 AS sum(number_sales)#201] +Aggregate Attributes [2]: [sum(sum_sales#114)#162, sum(number_sales#115)#163] +Results [6]: [null AS channel#164, null AS i_brand_id#165, null AS i_class_id#166, null AS i_category_id#167, sum(sum_sales#114)#162 AS sum(sum_sales)#168, sum(number_sales#115)#163 AS sum(number_sales)#169] (123) Union @@ -726,7 +726,7 @@ Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sa (125) Exchange Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] -Arguments: hashpartitioning(channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115, 5), ENSURE_REQUIREMENTS, [id=#202] +Arguments: hashpartitioning(channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115, 5), ENSURE_REQUIREMENTS, [id=#170] (126) HashAggregate [codegen id : 406] Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] @@ -764,94 +764,94 @@ Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (128) Scan parquet default.store_sales -Output [3]: [ss_quantity#203, ss_list_price#204, ss_sold_date_sk#205] +Output [3]: [ss_quantity#171, ss_list_price#172, ss_sold_date_sk#173] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#205), dynamicpruningexpression(ss_sold_date_sk#205 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#173), dynamicpruningexpression(ss_sold_date_sk#173 IN dynamicpruning#12)] ReadSchema: struct (129) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#203, ss_list_price#204, ss_sold_date_sk#205] +Input [3]: [ss_quantity#171, ss_list_price#172, ss_sold_date_sk#173] (130) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#206] +Output [1]: [d_date_sk#174] (131) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#205] -Right keys [1]: [d_date_sk#206] +Left keys [1]: [ss_sold_date_sk#173] +Right keys [1]: [d_date_sk#174] Join condition: None (132) Project [codegen id : 2] -Output [2]: [ss_quantity#203 AS quantity#207, ss_list_price#204 AS list_price#208] -Input [4]: [ss_quantity#203, ss_list_price#204, ss_sold_date_sk#205, d_date_sk#206] +Output [2]: [ss_quantity#171 AS quantity#175, ss_list_price#172 AS list_price#176] +Input [4]: [ss_quantity#171, ss_list_price#172, ss_sold_date_sk#173, d_date_sk#174] (133) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#209, cs_list_price#210, cs_sold_date_sk#211] +Output [3]: [cs_quantity#177, cs_list_price#178, cs_sold_date_sk#179] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#211), dynamicpruningexpression(cs_sold_date_sk#211 IN dynamicpruning#212)] +PartitionFilters: [isnotnull(cs_sold_date_sk#179), dynamicpruningexpression(cs_sold_date_sk#179 IN dynamicpruning#180)] ReadSchema: struct (134) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#209, cs_list_price#210, cs_sold_date_sk#211] +Input [3]: [cs_quantity#177, cs_list_price#178, cs_sold_date_sk#179] (135) ReusedExchange [Reuses operator id: 151] -Output [1]: [d_date_sk#213] +Output [1]: [d_date_sk#181] (136) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#211] -Right keys [1]: [d_date_sk#213] +Left keys [1]: [cs_sold_date_sk#179] +Right keys [1]: [d_date_sk#181] Join condition: None (137) Project [codegen id : 4] -Output [2]: [cs_quantity#209 AS quantity#214, cs_list_price#210 AS list_price#215] -Input [4]: [cs_quantity#209, cs_list_price#210, cs_sold_date_sk#211, d_date_sk#213] +Output [2]: [cs_quantity#177 AS quantity#182, cs_list_price#178 AS list_price#183] +Input [4]: [cs_quantity#177, cs_list_price#178, cs_sold_date_sk#179, d_date_sk#181] (138) Scan parquet default.web_sales -Output [3]: [ws_quantity#216, ws_list_price#217, ws_sold_date_sk#218] +Output [3]: [ws_quantity#184, ws_list_price#185, ws_sold_date_sk#186] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#218), dynamicpruningexpression(ws_sold_date_sk#218 IN dynamicpruning#212)] +PartitionFilters: [isnotnull(ws_sold_date_sk#186), dynamicpruningexpression(ws_sold_date_sk#186 IN dynamicpruning#180)] ReadSchema: struct (139) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#216, ws_list_price#217, ws_sold_date_sk#218] +Input [3]: [ws_quantity#184, ws_list_price#185, ws_sold_date_sk#186] (140) ReusedExchange [Reuses operator id: 151] -Output [1]: [d_date_sk#219] +Output [1]: [d_date_sk#187] (141) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#218] -Right keys [1]: [d_date_sk#219] +Left keys [1]: [ws_sold_date_sk#186] +Right keys [1]: [d_date_sk#187] Join condition: None (142) Project [codegen id : 6] -Output [2]: [ws_quantity#216 AS quantity#220, ws_list_price#217 AS list_price#221] -Input [4]: [ws_quantity#216, ws_list_price#217, ws_sold_date_sk#218, d_date_sk#219] +Output [2]: [ws_quantity#184 AS quantity#188, ws_list_price#185 AS list_price#189] +Input [4]: [ws_quantity#184, ws_list_price#185, ws_sold_date_sk#186, d_date_sk#187] (143) Union (144) HashAggregate [codegen id : 7] -Input [2]: [quantity#207, list_price#208] +Input [2]: [quantity#175, list_price#176] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#207 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#208 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#222, count#223] -Results [2]: [sum#224, count#225] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#175 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#190, count#191] +Results [2]: [sum#192, count#193] (145) Exchange -Input [2]: [sum#224, count#225] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#226] +Input [2]: [sum#192, count#193] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#194] (146) HashAggregate [codegen id : 8] -Input [2]: [sum#224, count#225] +Input [2]: [sum#192, count#193] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#207 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#208 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#207 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#208 as decimal(12,2)))), DecimalType(18,2), true))#227] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#207 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#208 as decimal(12,2)))), DecimalType(18,2), true))#227 AS average_sales#228] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#175 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#175 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2), true))#195] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#175 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#176 as decimal(12,2)))), DecimalType(18,2), true))#195 AS average_sales#196] -Subquery:2 Hosting operator id = 128 Hosting Expression = ss_sold_date_sk#205 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 128 Hosting Expression = ss_sold_date_sk#173 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 133 Hosting Expression = cs_sold_date_sk#211 IN dynamicpruning#212 +Subquery:3 Hosting operator id = 133 Hosting Expression = cs_sold_date_sk#179 IN dynamicpruning#180 BroadcastExchange (151) +- * Project (150) +- * Filter (149) @@ -860,28 +860,28 @@ BroadcastExchange (151) (147) Scan parquet default.date_dim -Output [2]: [d_date_sk#213, d_year#229] +Output [2]: [d_date_sk#181, d_year#197] 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 (148) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#213, d_year#229] +Input [2]: [d_date_sk#181, d_year#197] (149) Filter [codegen id : 1] -Input [2]: [d_date_sk#213, d_year#229] -Condition : (((isnotnull(d_year#229) AND (d_year#229 >= 1998)) AND (d_year#229 <= 2000)) AND isnotnull(d_date_sk#213)) +Input [2]: [d_date_sk#181, d_year#197] +Condition : (((isnotnull(d_year#197) AND (d_year#197 >= 1998)) AND (d_year#197 <= 2000)) AND isnotnull(d_date_sk#181)) (150) Project [codegen id : 1] -Output [1]: [d_date_sk#213] -Input [2]: [d_date_sk#213, d_year#229] +Output [1]: [d_date_sk#181] +Input [2]: [d_date_sk#181, d_year#197] (151) BroadcastExchange -Input [1]: [d_date_sk#213] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#230] +Input [1]: [d_date_sk#181] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#198] -Subquery:4 Hosting operator id = 138 Hosting Expression = ws_sold_date_sk#218 IN dynamicpruning#212 +Subquery:4 Hosting operator id = 138 Hosting Expression = ws_sold_date_sk#186 IN dynamicpruning#180 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (156) @@ -892,26 +892,26 @@ BroadcastExchange (156) (152) Scan parquet default.date_dim -Output [3]: [d_date_sk#48, d_year#231, d_moy#232] +Output [3]: [d_date_sk#48, d_year#199, d_moy#200] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#48, d_year#231, d_moy#232] +Input [3]: [d_date_sk#48, d_year#199, d_moy#200] (154) Filter [codegen id : 1] -Input [3]: [d_date_sk#48, d_year#231, d_moy#232] -Condition : ((((isnotnull(d_year#231) AND isnotnull(d_moy#232)) AND (d_year#231 = 2000)) AND (d_moy#232 = 11)) AND isnotnull(d_date_sk#48)) +Input [3]: [d_date_sk#48, d_year#199, d_moy#200] +Condition : ((((isnotnull(d_year#199) AND isnotnull(d_moy#200)) AND (d_year#199 = 2000)) AND (d_moy#200 = 11)) AND isnotnull(d_date_sk#48)) (155) Project [codegen id : 1] Output [1]: [d_date_sk#48] -Input [3]: [d_date_sk#48, d_year#231, d_moy#232] +Input [3]: [d_date_sk#48, d_year#199, d_moy#200] (156) BroadcastExchange Input [1]: [d_date_sk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#201] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (161) @@ -922,26 +922,26 @@ BroadcastExchange (161) (157) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#234] +Output [2]: [d_date_sk#27, d_year#202] 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 (158) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#234] +Input [2]: [d_date_sk#27, d_year#202] (159) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#234] -Condition : (((isnotnull(d_year#234) AND (d_year#234 >= 1999)) AND (d_year#234 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#27, d_year#202] +Condition : (((isnotnull(d_year#202) AND (d_year#202 >= 1999)) AND (d_year#202 <= 2001)) AND isnotnull(d_date_sk#27)) (160) Project [codegen id : 1] Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#234] +Input [2]: [d_date_sk#27, d_year#202] (161) BroadcastExchange Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#203] Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt index b7c1f6d27b0ea..506e18eabcc20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt @@ -370,484 +370,484 @@ Aggregate Attributes [7]: [avg(agg1#35)#71, avg(agg2#36)#72, avg(agg3#37)#73, av Results [11]: [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, avg(agg1#35)#71 AS agg1#78, avg(agg2#36)#72 AS agg2#79, avg(agg3#37)#73 AS agg3#80, avg(agg4#38)#74 AS agg4#81, avg(agg5#39)#75 AS agg5#82, avg(agg6#40)#76 AS agg6#83, avg(agg7#41)#77 AS agg7#84] (48) ReusedExchange [Reuses operator id: 20] -Output [8]: [cs_bill_customer_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#91, i_item_id#92] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#18] (49) Sort [codegen id : 19] -Input [8]: [cs_bill_customer_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#91, i_item_id#92] -Arguments: [cs_bill_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#18] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (50) Scan parquet default.customer -Output [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_current_addr_sk#95, c_birth_month#96, c_birth_year#97] +Output [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_current_addr_sk#95, c_birth_month#96, c_birth_year#97] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] (52) Filter [codegen id : 21] -Input [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_current_addr_sk#95, c_birth_month#96, c_birth_year#97] -Condition : (((c_birth_month#96 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#93)) AND isnotnull(c_current_cdemo_sk#94)) AND isnotnull(c_current_addr_sk#95)) +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] +Condition : (((c_birth_month#24 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#21)) AND isnotnull(c_current_cdemo_sk#22)) AND isnotnull(c_current_addr_sk#23)) (53) Project [codegen id : 21] -Output [4]: [c_customer_sk#93, c_current_cdemo_sk#94, c_current_addr_sk#95, c_birth_year#97] -Input [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_current_addr_sk#95, c_birth_month#96, c_birth_year#97] +Output [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_year#25] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] (54) Scan parquet default.customer_address -Output [3]: [ca_address_sk#98, ca_state#99, ca_country#100] +Output [3]: [ca_address_sk#26, ca_state#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#98, ca_state#99, ca_country#100] +Input [3]: [ca_address_sk#26, ca_state#28, ca_country#29] (56) Filter [codegen id : 20] -Input [3]: [ca_address_sk#98, ca_state#99, ca_country#100] -Condition : (ca_state#99 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#98)) +Input [3]: [ca_address_sk#26, ca_state#28, ca_country#29] +Condition : (ca_state#28 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#26)) (57) BroadcastExchange -Input [3]: [ca_address_sk#98, ca_state#99, ca_country#100] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#101] +Input [3]: [ca_address_sk#26, ca_state#28, ca_country#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] (58) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#95] -Right keys [1]: [ca_address_sk#98] +Left keys [1]: [c_current_addr_sk#23] +Right keys [1]: [ca_address_sk#26] Join condition: None (59) Project [codegen id : 21] -Output [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_birth_year#97, ca_state#99, ca_country#100] -Input [7]: [c_customer_sk#93, c_current_cdemo_sk#94, c_current_addr_sk#95, c_birth_year#97, ca_address_sk#98, ca_state#99, ca_country#100] +Output [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_state#28, ca_country#29] +Input [7]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_year#25, ca_address_sk#26, ca_state#28, ca_country#29] (60) Exchange -Input [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_birth_year#97, ca_state#99, ca_country#100] -Arguments: hashpartitioning(c_current_cdemo_sk#94, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_state#28, ca_country#29] +Arguments: hashpartitioning(c_current_cdemo_sk#22, 5), ENSURE_REQUIREMENTS, [id=#86] (61) Sort [codegen id : 22] -Input [5]: [c_customer_sk#93, c_current_cdemo_sk#94, c_birth_year#97, ca_state#99, ca_country#100] -Arguments: [c_current_cdemo_sk#94 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_state#28, ca_country#29] +Arguments: [c_current_cdemo_sk#22 ASC NULLS FIRST], false, 0 (62) ReusedExchange [Reuses operator id: 37] -Output [1]: [cd_demo_sk#103] +Output [1]: [cd_demo_sk#32] (63) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#103] -Arguments: [cd_demo_sk#103 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#32] +Arguments: [cd_demo_sk#32 ASC NULLS FIRST], false, 0 (64) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#94] -Right keys [1]: [cd_demo_sk#103] +Left keys [1]: [c_current_cdemo_sk#22] +Right keys [1]: [cd_demo_sk#32] Join condition: None (65) Project [codegen id : 25] -Output [4]: [c_customer_sk#93, c_birth_year#97, ca_state#99, ca_country#100] -Input [6]: [c_customer_sk#93, c_current_cdemo_sk#94, c_birth_year#97, ca_state#99, ca_country#100, cd_demo_sk#103] +Output [4]: [c_customer_sk#21, c_birth_year#25, ca_state#28, ca_country#29] +Input [6]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_state#28, ca_country#29, cd_demo_sk#32] (66) Exchange -Input [4]: [c_customer_sk#93, c_birth_year#97, ca_state#99, ca_country#100] -Arguments: hashpartitioning(c_customer_sk#93, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [4]: [c_customer_sk#21, c_birth_year#25, ca_state#28, ca_country#29] +Arguments: hashpartitioning(c_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#87] (67) Sort [codegen id : 26] -Input [4]: [c_customer_sk#93, c_birth_year#97, ca_state#99, ca_country#100] -Arguments: [c_customer_sk#93 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#21, c_birth_year#25, ca_state#28, ca_country#29] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#85] -Right keys [1]: [c_customer_sk#93] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#21] Join condition: None (69) Project [codegen id : 27] -Output [10]: [i_item_id#92, ca_country#100, ca_state#99, cast(cs_quantity#86 as decimal(12,2)) AS agg1#35, cast(cs_list_price#87 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#39, cast(c_birth_year#97 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#91 as decimal(12,2)) AS agg7#41] -Input [12]: [cs_bill_customer_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#91, i_item_id#92, c_customer_sk#93, c_birth_year#97, ca_state#99, ca_country#100] +Output [10]: [i_item_id#18, ca_country#29, ca_state#28, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#25 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +Input [12]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#18, c_customer_sk#21, c_birth_year#25, ca_state#28, ca_country#29] (70) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#92, ca_country#100, ca_state#99, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#92, ca_country#100, ca_state#99] +Input [10]: [i_item_id#18, ca_country#29, ca_state#28, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [3]: [i_item_id#18, ca_country#29, ca_state#28] Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] -Aggregate Attributes [14]: [sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] -Results [17]: [i_item_id#92, ca_country#100, ca_state#99, sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132] +Aggregate Attributes [14]: [sum#88, count#89, sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101] +Results [17]: [i_item_id#18, ca_country#29, ca_state#28, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115] (71) Exchange -Input [17]: [i_item_id#92, ca_country#100, ca_state#99, sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132] -Arguments: hashpartitioning(i_item_id#92, ca_country#100, ca_state#99, 5), ENSURE_REQUIREMENTS, [id=#133] +Input [17]: [i_item_id#18, ca_country#29, ca_state#28, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115] +Arguments: hashpartitioning(i_item_id#18, ca_country#29, ca_state#28, 5), ENSURE_REQUIREMENTS, [id=#116] (72) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#92, ca_country#100, ca_state#99, sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132] -Keys [3]: [i_item_id#92, ca_country#100, ca_state#99] +Input [17]: [i_item_id#18, ca_country#29, ca_state#28, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115] +Keys [3]: [i_item_id#18, ca_country#29, ca_state#28] Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] -Aggregate Attributes [7]: [avg(agg1#35)#134, avg(agg2#36)#135, avg(agg3#37)#136, avg(agg4#38)#137, avg(agg5#39)#138, avg(agg6#40)#139, avg(agg7#41)#140] -Results [11]: [i_item_id#92, ca_country#100, ca_state#99, null AS county#141, avg(agg1#35)#134 AS agg1#142, avg(agg2#36)#135 AS agg2#143, avg(agg3#37)#136 AS agg3#144, avg(agg4#38)#137 AS agg4#145, avg(agg5#39)#138 AS agg5#146, avg(agg6#40)#139 AS agg6#147, avg(agg7#41)#140 AS agg7#148] +Aggregate Attributes [7]: [avg(agg1#35)#117, avg(agg2#36)#118, avg(agg3#37)#119, avg(agg4#38)#120, avg(agg5#39)#121, avg(agg6#40)#122, avg(agg7#41)#123] +Results [11]: [i_item_id#18, ca_country#29, ca_state#28, null AS county#124, avg(agg1#35)#117 AS agg1#125, avg(agg2#36)#118 AS agg2#126, avg(agg3#37)#119 AS agg3#127, avg(agg4#38)#120 AS agg4#128, avg(agg5#39)#121 AS agg5#129, avg(agg6#40)#122 AS agg6#130, avg(agg7#41)#123 AS agg7#131] (73) ReusedExchange [Reuses operator id: 20] -Output [8]: [cs_bill_customer_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#155, i_item_id#156] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#18] (74) Sort [codegen id : 33] -Input [8]: [cs_bill_customer_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#155, i_item_id#156] -Arguments: [cs_bill_customer_sk#149 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#18] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (75) Scan parquet default.customer -Output [5]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_month#160, c_birth_year#161] +Output [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_month#160, c_birth_year#161] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] (77) Filter [codegen id : 35] -Input [5]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_month#160, c_birth_year#161] -Condition : (((c_birth_month#160 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#157)) AND isnotnull(c_current_cdemo_sk#158)) AND isnotnull(c_current_addr_sk#159)) +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] +Condition : (((c_birth_month#24 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#21)) AND isnotnull(c_current_cdemo_sk#22)) AND isnotnull(c_current_addr_sk#23)) (78) Project [codegen id : 35] -Output [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#161] -Input [5]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_month#160, c_birth_year#161] +Output [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_year#25] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] (79) Scan parquet default.customer_address -Output [3]: [ca_address_sk#162, ca_state#163, ca_country#164] +Output [3]: [ca_address_sk#26, ca_state#28, ca_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#162, ca_state#163, ca_country#164] +Input [3]: [ca_address_sk#26, ca_state#28, ca_country#29] (81) Filter [codegen id : 34] -Input [3]: [ca_address_sk#162, ca_state#163, ca_country#164] -Condition : (ca_state#163 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#162)) +Input [3]: [ca_address_sk#26, ca_state#28, ca_country#29] +Condition : (ca_state#28 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#26)) (82) Project [codegen id : 34] -Output [2]: [ca_address_sk#162, ca_country#164] -Input [3]: [ca_address_sk#162, ca_state#163, ca_country#164] +Output [2]: [ca_address_sk#26, ca_country#29] +Input [3]: [ca_address_sk#26, ca_state#28, ca_country#29] (83) BroadcastExchange -Input [2]: [ca_address_sk#162, ca_country#164] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#165] +Input [2]: [ca_address_sk#26, ca_country#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] (84) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#159] -Right keys [1]: [ca_address_sk#162] +Left keys [1]: [c_current_addr_sk#23] +Right keys [1]: [ca_address_sk#26] Join condition: None (85) Project [codegen id : 35] -Output [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_birth_year#161, ca_country#164] -Input [6]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#161, ca_address_sk#162, ca_country#164] +Output [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_country#29] +Input [6]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_year#25, ca_address_sk#26, ca_country#29] (86) Exchange -Input [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_birth_year#161, ca_country#164] -Arguments: hashpartitioning(c_current_cdemo_sk#158, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_country#29] +Arguments: hashpartitioning(c_current_cdemo_sk#22, 5), ENSURE_REQUIREMENTS, [id=#133] (87) Sort [codegen id : 36] -Input [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_birth_year#161, ca_country#164] -Arguments: [c_current_cdemo_sk#158 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_country#29] +Arguments: [c_current_cdemo_sk#22 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 37] -Output [1]: [cd_demo_sk#167] +Output [1]: [cd_demo_sk#32] (89) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#167] -Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#32] +Arguments: [cd_demo_sk#32 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#158] -Right keys [1]: [cd_demo_sk#167] +Left keys [1]: [c_current_cdemo_sk#22] +Right keys [1]: [cd_demo_sk#32] Join condition: None (91) Project [codegen id : 39] -Output [3]: [c_customer_sk#157, c_birth_year#161, ca_country#164] -Input [5]: [c_customer_sk#157, c_current_cdemo_sk#158, c_birth_year#161, ca_country#164, cd_demo_sk#167] +Output [3]: [c_customer_sk#21, c_birth_year#25, ca_country#29] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, ca_country#29, cd_demo_sk#32] (92) Exchange -Input [3]: [c_customer_sk#157, c_birth_year#161, ca_country#164] -Arguments: hashpartitioning(c_customer_sk#157, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [3]: [c_customer_sk#21, c_birth_year#25, ca_country#29] +Arguments: hashpartitioning(c_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#134] (93) Sort [codegen id : 40] -Input [3]: [c_customer_sk#157, c_birth_year#161, ca_country#164] -Arguments: [c_customer_sk#157 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#21, c_birth_year#25, ca_country#29] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#149] -Right keys [1]: [c_customer_sk#157] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#21] Join condition: None (95) Project [codegen id : 41] -Output [9]: [i_item_id#156, ca_country#164, cast(cs_quantity#150 as decimal(12,2)) AS agg1#35, cast(cs_list_price#151 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#153 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#152 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#154 as decimal(12,2)) AS agg5#39, cast(c_birth_year#161 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#155 as decimal(12,2)) AS agg7#41] -Input [11]: [cs_bill_customer_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#155, i_item_id#156, c_customer_sk#157, c_birth_year#161, ca_country#164] +Output [9]: [i_item_id#18, ca_country#29, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#25 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +Input [11]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#18, c_customer_sk#21, c_birth_year#25, ca_country#29] (96) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#156, ca_country#164, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#156, ca_country#164] +Input [9]: [i_item_id#18, ca_country#29, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [2]: [i_item_id#18, ca_country#29] Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] -Aggregate Attributes [14]: [sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182] -Results [16]: [i_item_id#156, ca_country#164, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] +Aggregate Attributes [14]: [sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146, sum#147, count#148] +Results [16]: [i_item_id#18, ca_country#29, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] (97) Exchange -Input [16]: [i_item_id#156, ca_country#164, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] -Arguments: hashpartitioning(i_item_id#156, ca_country#164, 5), ENSURE_REQUIREMENTS, [id=#197] +Input [16]: [i_item_id#18, ca_country#29, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] +Arguments: hashpartitioning(i_item_id#18, ca_country#29, 5), ENSURE_REQUIREMENTS, [id=#163] (98) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#156, ca_country#164, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] -Keys [2]: [i_item_id#156, ca_country#164] +Input [16]: [i_item_id#18, ca_country#29, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] +Keys [2]: [i_item_id#18, ca_country#29] Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] -Aggregate Attributes [7]: [avg(agg1#35)#198, avg(agg2#36)#199, avg(agg3#37)#200, avg(agg4#38)#201, avg(agg5#39)#202, avg(agg6#40)#203, avg(agg7#41)#204] -Results [11]: [i_item_id#156, ca_country#164, null AS ca_state#205, null AS county#206, avg(agg1#35)#198 AS agg1#207, avg(agg2#36)#199 AS agg2#208, avg(agg3#37)#200 AS agg3#209, avg(agg4#38)#201 AS agg4#210, avg(agg5#39)#202 AS agg5#211, avg(agg6#40)#203 AS agg6#212, avg(agg7#41)#204 AS agg7#213] +Aggregate Attributes [7]: [avg(agg1#35)#164, avg(agg2#36)#165, avg(agg3#37)#166, avg(agg4#38)#167, avg(agg5#39)#168, avg(agg6#40)#169, avg(agg7#41)#170] +Results [11]: [i_item_id#18, ca_country#29, null AS ca_state#171, null AS county#172, avg(agg1#35)#164 AS agg1#173, avg(agg2#36)#165 AS agg2#174, avg(agg3#37)#166 AS agg3#175, avg(agg4#38)#167 AS agg4#176, avg(agg5#39)#168 AS agg5#177, avg(agg6#40)#169 AS agg6#178, avg(agg7#41)#170 AS agg7#179] (99) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#214, cs_bill_cdemo_sk#215, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cs_sold_date_sk#222] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#222), dynamicpruningexpression(cs_sold_date_sk#222 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 49] -Input [9]: [cs_bill_customer_sk#214, cs_bill_cdemo_sk#215, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cs_sold_date_sk#222] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (101) Filter [codegen id : 49] -Input [9]: [cs_bill_customer_sk#214, cs_bill_cdemo_sk#215, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cs_sold_date_sk#222] -Condition : ((isnotnull(cs_bill_cdemo_sk#215) AND isnotnull(cs_bill_customer_sk#214)) AND isnotnull(cs_item_sk#216)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (102) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#223, cd_dep_count#224] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (103) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#215] -Right keys [1]: [cd_demo_sk#223] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (104) Project [codegen id : 49] -Output [9]: [cs_bill_customer_sk#214, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cs_sold_date_sk#222, cd_dep_count#224] -Input [11]: [cs_bill_customer_sk#214, cs_bill_cdemo_sk#215, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cs_sold_date_sk#222, cd_demo_sk#223, cd_dep_count#224] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (105) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#225] +Output [1]: [d_date_sk#16] (106) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#222] -Right keys [1]: [d_date_sk#225] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (107) Project [codegen id : 49] -Output [8]: [cs_bill_customer_sk#214, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cd_dep_count#224] -Input [10]: [cs_bill_customer_sk#214, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cs_sold_date_sk#222, cd_dep_count#224, d_date_sk#225] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (108) Scan parquet default.customer -Output [5]: [c_customer_sk#226, c_current_cdemo_sk#227, c_current_addr_sk#228, c_birth_month#229, c_birth_year#230] +Output [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (109) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#226, c_current_cdemo_sk#227, c_current_addr_sk#228, c_birth_month#229, c_birth_year#230] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] (110) Filter [codegen id : 46] -Input [5]: [c_customer_sk#226, c_current_cdemo_sk#227, c_current_addr_sk#228, c_birth_month#229, c_birth_year#230] -Condition : (((c_birth_month#229 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#226)) AND isnotnull(c_current_cdemo_sk#227)) AND isnotnull(c_current_addr_sk#228)) +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] +Condition : (((c_birth_month#24 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#21)) AND isnotnull(c_current_cdemo_sk#22)) AND isnotnull(c_current_addr_sk#23)) (111) Project [codegen id : 46] -Output [4]: [c_customer_sk#226, c_current_cdemo_sk#227, c_current_addr_sk#228, c_birth_year#230] -Input [5]: [c_customer_sk#226, c_current_cdemo_sk#227, c_current_addr_sk#228, c_birth_month#229, c_birth_year#230] +Output [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_year#25] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_month#24, c_birth_year#25] (112) Scan parquet default.customer_address -Output [2]: [ca_address_sk#231, ca_state#232] +Output [2]: [ca_address_sk#26, ca_state#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#231, ca_state#232] +Input [2]: [ca_address_sk#26, ca_state#28] (114) Filter [codegen id : 45] -Input [2]: [ca_address_sk#231, ca_state#232] -Condition : (ca_state#232 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#231)) +Input [2]: [ca_address_sk#26, ca_state#28] +Condition : (ca_state#28 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#26)) (115) Project [codegen id : 45] -Output [1]: [ca_address_sk#231] -Input [2]: [ca_address_sk#231, ca_state#232] +Output [1]: [ca_address_sk#26] +Input [2]: [ca_address_sk#26, ca_state#28] (116) BroadcastExchange -Input [1]: [ca_address_sk#231] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#233] +Input [1]: [ca_address_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#180] (117) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#228] -Right keys [1]: [ca_address_sk#231] +Left keys [1]: [c_current_addr_sk#23] +Right keys [1]: [ca_address_sk#26] Join condition: None (118) Project [codegen id : 46] -Output [3]: [c_customer_sk#226, c_current_cdemo_sk#227, c_birth_year#230] -Input [5]: [c_customer_sk#226, c_current_cdemo_sk#227, c_current_addr_sk#228, c_birth_year#230, ca_address_sk#231] +Output [3]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25] +Input [5]: [c_customer_sk#21, c_current_cdemo_sk#22, c_current_addr_sk#23, c_birth_year#25, ca_address_sk#26] (119) BroadcastExchange -Input [3]: [c_customer_sk#226, c_current_cdemo_sk#227, c_birth_year#230] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#234] +Input [3]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#181] (120) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#235] +Output [1]: [cd_demo_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (121) ColumnarToRow -Input [1]: [cd_demo_sk#235] +Input [1]: [cd_demo_sk#32] (122) Filter -Input [1]: [cd_demo_sk#235] -Condition : isnotnull(cd_demo_sk#235) +Input [1]: [cd_demo_sk#32] +Condition : isnotnull(cd_demo_sk#32) (123) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#227] -Right keys [1]: [cd_demo_sk#235] +Left keys [1]: [c_current_cdemo_sk#22] +Right keys [1]: [cd_demo_sk#32] Join condition: None (124) Project [codegen id : 47] -Output [2]: [c_customer_sk#226, c_birth_year#230] -Input [4]: [c_customer_sk#226, c_current_cdemo_sk#227, c_birth_year#230, cd_demo_sk#235] +Output [2]: [c_customer_sk#21, c_birth_year#25] +Input [4]: [c_customer_sk#21, c_current_cdemo_sk#22, c_birth_year#25, cd_demo_sk#32] (125) BroadcastExchange -Input [2]: [c_customer_sk#226, c_birth_year#230] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#236] +Input [2]: [c_customer_sk#21, c_birth_year#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#182] (126) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#214] -Right keys [1]: [c_customer_sk#226] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#21] Join condition: None (127) Project [codegen id : 49] -Output [8]: [cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cd_dep_count#224, c_birth_year#230] -Input [10]: [cs_bill_customer_sk#214, cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cd_dep_count#224, c_customer_sk#226, c_birth_year#230] +Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#25] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_customer_sk#21, c_birth_year#25] (128) ReusedExchange [Reuses operator id: 17] -Output [2]: [i_item_sk#237, i_item_id#238] +Output [2]: [i_item_sk#17, i_item_id#18] (129) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#216] -Right keys [1]: [i_item_sk#237] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#17] Join condition: None (130) Project [codegen id : 49] -Output [8]: [i_item_id#238, cast(cs_quantity#217 as decimal(12,2)) AS agg1#35, cast(cs_list_price#218 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#220 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#219 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#221 as decimal(12,2)) AS agg5#39, cast(c_birth_year#230 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#224 as decimal(12,2)) AS agg7#41] -Input [10]: [cs_item_sk#216, cs_quantity#217, cs_list_price#218, cs_sales_price#219, cs_coupon_amt#220, cs_net_profit#221, cd_dep_count#224, c_birth_year#230, i_item_sk#237, i_item_id#238] +Output [8]: [i_item_id#18, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#25 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#25, i_item_sk#17, i_item_id#18] (131) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#238, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#238] +Input [8]: [i_item_id#18, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [1]: [i_item_id#18] Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] -Aggregate Attributes [14]: [sum#239, count#240, sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252] -Results [15]: [i_item_id#238, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266] +Aggregate Attributes [14]: [sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] +Results [15]: [i_item_id#18, sum#197, count#198, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210] (132) Exchange -Input [15]: [i_item_id#238, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266] -Arguments: hashpartitioning(i_item_id#238, 5), ENSURE_REQUIREMENTS, [id=#267] +Input [15]: [i_item_id#18, sum#197, count#198, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [id=#211] (133) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#238, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266] -Keys [1]: [i_item_id#238] +Input [15]: [i_item_id#18, sum#197, count#198, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210] +Keys [1]: [i_item_id#18] Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] -Aggregate Attributes [7]: [avg(agg1#35)#268, avg(agg2#36)#269, avg(agg3#37)#270, avg(agg4#38)#271, avg(agg5#39)#272, avg(agg6#40)#273, avg(agg7#41)#274] -Results [11]: [i_item_id#238, null AS ca_country#275, null AS ca_state#276, null AS county#277, avg(agg1#35)#268 AS agg1#278, avg(agg2#36)#269 AS agg2#279, avg(agg3#37)#270 AS agg3#280, avg(agg4#38)#271 AS agg4#281, avg(agg5#39)#272 AS agg5#282, avg(agg6#40)#273 AS agg6#283, avg(agg7#41)#274 AS agg7#284] +Aggregate Attributes [7]: [avg(agg1#35)#212, avg(agg2#36)#213, avg(agg3#37)#214, avg(agg4#38)#215, avg(agg5#39)#216, avg(agg6#40)#217, avg(agg7#41)#218] +Results [11]: [i_item_id#18, null AS ca_country#219, null AS ca_state#220, null AS county#221, avg(agg1#35)#212 AS agg1#222, avg(agg2#36)#213 AS agg2#223, avg(agg3#37)#214 AS agg3#224, avg(agg4#38)#215 AS agg4#225, avg(agg5#39)#216 AS agg5#226, avg(agg6#40)#217 AS agg6#227, avg(agg7#41)#218 AS agg7#228] (134) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#285, cs_bill_cdemo_sk#286, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cs_sold_date_sk#293] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#293), dynamicpruningexpression(cs_sold_date_sk#293 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 57] -Input [9]: [cs_bill_customer_sk#285, cs_bill_cdemo_sk#286, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cs_sold_date_sk#293] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (136) Filter [codegen id : 57] -Input [9]: [cs_bill_customer_sk#285, cs_bill_cdemo_sk#286, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cs_sold_date_sk#293] -Condition : ((isnotnull(cs_bill_cdemo_sk#286) AND isnotnull(cs_bill_customer_sk#285)) AND isnotnull(cs_item_sk#287)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (137) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#294, cd_dep_count#295] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (138) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#286] -Right keys [1]: [cd_demo_sk#294] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (139) Project [codegen id : 57] -Output [9]: [cs_bill_customer_sk#285, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cs_sold_date_sk#293, cd_dep_count#295] -Input [11]: [cs_bill_customer_sk#285, cs_bill_cdemo_sk#286, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cs_sold_date_sk#293, cd_demo_sk#294, cd_dep_count#295] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (140) ReusedExchange [Reuses operator id: 161] -Output [1]: [d_date_sk#296] +Output [1]: [d_date_sk#16] (141) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#293] -Right keys [1]: [d_date_sk#296] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (142) Project [codegen id : 57] -Output [8]: [cs_bill_customer_sk#285, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cd_dep_count#295] -Input [10]: [cs_bill_customer_sk#285, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cs_sold_date_sk#293, cd_dep_count#295, d_date_sk#296] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (143) Scan parquet default.item -Output [1]: [i_item_sk#297] +Output [1]: [i_item_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#297] +Input [1]: [i_item_sk#17] (145) Filter [codegen id : 53] -Input [1]: [i_item_sk#297] -Condition : isnotnull(i_item_sk#297) +Input [1]: [i_item_sk#17] +Condition : isnotnull(i_item_sk#17) (146) BroadcastExchange -Input [1]: [i_item_sk#297] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#298] +Input [1]: [i_item_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#229] (147) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#287] -Right keys [1]: [i_item_sk#297] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#17] Join condition: None (148) Project [codegen id : 57] -Output [7]: [cs_bill_customer_sk#285, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cd_dep_count#295] -Input [9]: [cs_bill_customer_sk#285, cs_item_sk#287, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cd_dep_count#295, i_item_sk#297] +Output [7]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#17] (149) ReusedExchange [Reuses operator id: 125] -Output [2]: [c_customer_sk#299, c_birth_year#300] +Output [2]: [c_customer_sk#21, c_birth_year#25] (150) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#285] -Right keys [1]: [c_customer_sk#299] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#21] Join condition: None (151) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#288 as decimal(12,2)) AS agg1#35, cast(cs_list_price#289 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#291 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#290 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#292 as decimal(12,2)) AS agg5#39, cast(c_birth_year#300 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#295 as decimal(12,2)) AS agg7#41] -Input [9]: [cs_bill_customer_sk#285, cs_quantity#288, cs_list_price#289, cs_sales_price#290, cs_coupon_amt#291, cs_net_profit#292, cd_dep_count#295, c_customer_sk#299, c_birth_year#300] +Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#25 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +Input [9]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_customer_sk#21, c_birth_year#25] (152) HashAggregate [codegen id : 57] Input [7]: [agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] Keys: [] Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] -Aggregate Attributes [14]: [sum#301, count#302, sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314] -Results [14]: [sum#315, count#316, sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328] +Aggregate Attributes [14]: [sum#230, count#231, sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243] +Results [14]: [sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257] (153) Exchange -Input [14]: [sum#315, count#316, sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#329] +Input [14]: [sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#258] (154) HashAggregate [codegen id : 58] -Input [14]: [sum#315, count#316, sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328] +Input [14]: [sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257] Keys: [] Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] -Aggregate Attributes [7]: [avg(agg1#35)#330, avg(agg2#36)#331, avg(agg3#37)#332, avg(agg4#38)#333, avg(agg5#39)#334, avg(agg6#40)#335, avg(agg7#41)#336] -Results [11]: [null AS i_item_id#337, null AS ca_country#338, null AS ca_state#339, null AS county#340, avg(agg1#35)#330 AS agg1#341, avg(agg2#36)#331 AS agg2#342, avg(agg3#37)#332 AS agg3#343, avg(agg4#38)#333 AS agg4#344, avg(agg5#39)#334 AS agg5#345, avg(agg6#40)#335 AS agg6#346, avg(agg7#41)#336 AS agg7#347] +Aggregate Attributes [7]: [avg(agg1#35)#259, avg(agg2#36)#260, avg(agg3#37)#261, avg(agg4#38)#262, avg(agg5#39)#263, avg(agg6#40)#264, avg(agg7#41)#265] +Results [11]: [null AS i_item_id#266, null AS ca_country#267, null AS ca_state#268, null AS county#269, avg(agg1#35)#259 AS agg1#270, avg(agg2#36)#260 AS agg2#271, avg(agg3#37)#261 AS agg3#272, avg(agg4#38)#262 AS agg4#273, avg(agg5#39)#263 AS agg5#274, avg(agg6#40)#264 AS agg6#275, avg(agg7#41)#265 AS agg7#276] (155) Union @@ -866,29 +866,29 @@ BroadcastExchange (161) (157) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_year#348] +Output [2]: [d_date_sk#16, d_year#277] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (158) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#16, d_year#348] +Input [2]: [d_date_sk#16, d_year#277] (159) Filter [codegen id : 1] -Input [2]: [d_date_sk#16, d_year#348] -Condition : ((isnotnull(d_year#348) AND (d_year#348 = 2001)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_year#277] +Condition : ((isnotnull(d_year#277) AND (d_year#277 = 2001)) AND isnotnull(d_date_sk#16)) (160) Project [codegen id : 1] Output [1]: [d_date_sk#16] -Input [2]: [d_date_sk#16, d_year#348] +Input [2]: [d_date_sk#16, d_year#277] (161) BroadcastExchange Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#349] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#278] -Subquery:2 Hosting operator id = 99 Hosting Expression = cs_sold_date_sk#222 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 99 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 134 Hosting Expression = cs_sold_date_sk#293 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 134 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 69c7a4c971931..7bd7a7ae80176 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -343,492 +343,492 @@ Aggregate Attributes [7]: [avg(agg1#33)#69, avg(agg2#34)#70, avg(agg3#35)#71, av Results [11]: [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, avg(agg1#33)#69 AS agg1#76, avg(agg2#34)#70 AS agg2#77, avg(agg3#35)#71 AS agg3#78, avg(agg4#36)#72 AS agg4#79, avg(agg5#37)#73 AS agg5#80, avg(agg6#38)#74 AS agg6#81, avg(agg7#39)#75 AS agg7#82] (42) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (44) Filter [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (45) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#92, cd_dep_count#93] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (46) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#84] -Right keys [1]: [cd_demo_sk#92] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (47) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] -Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (48) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#83] -Right keys [1]: [c_customer_sk#94] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (50) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (51) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#98] +Output [1]: [cd_demo_sk#22] (52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#98] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#22] Join condition: None (53) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20, cd_demo_sk#22] (54) Scan parquet default.customer_address -Output [3]: [ca_address_sk#99, ca_state#100, ca_country#101] +Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (56) Filter [codegen id : 12] -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Condition : (ca_state#100 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#99)) +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Condition : (ca_state#26 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#24)) (57) BroadcastExchange -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#102] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#83] (58) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#96] -Right keys [1]: [ca_address_sk#99] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (59) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#100, ca_country#101] -Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#100, ca_country#101] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20, ca_state#26, ca_country#27] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20, ca_address_sk#24, ca_state#26, ca_country#27] (60) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#29] (61) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (62) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#100, ca_country#101] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#100, ca_country#101, d_date_sk#103] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20, ca_state#26, ca_country#27] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20, ca_state#26, ca_country#27, d_date_sk#29] (63) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#104, i_item_id#105] +Output [2]: [i_item_sk#30, i_item_id#31] (64) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#85] -Right keys [1]: [i_item_sk#104] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#30] Join condition: None (65) Project [codegen id : 15] -Output [10]: [i_item_id#105, ca_country#101, ca_state#100, cast(cs_quantity#86 as decimal(12,2)) AS agg1#33, cast(cs_list_price#87 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#37, cast(c_birth_year#97 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#39] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#100, ca_country#101, i_item_sk#104, i_item_id#105] +Output [10]: [i_item_id#31, ca_country#27, ca_state#26, cast(cs_quantity#4 as decimal(12,2)) AS agg1#33, cast(cs_list_price#5 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#37, cast(c_birth_year#20 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#39] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20, ca_state#26, ca_country#27, i_item_sk#30, i_item_id#31] (66) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#105, ca_country#101, ca_state#100, agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] -Keys [3]: [i_item_id#105, ca_country#101, ca_state#100] +Input [10]: [i_item_id#31, ca_country#27, ca_state#26, agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] +Keys [3]: [i_item_id#31, ca_country#27, ca_state#26] Functions [7]: [partial_avg(agg1#33), partial_avg(agg2#34), partial_avg(agg3#35), partial_avg(agg4#36), partial_avg(agg5#37), partial_avg(agg6#38), partial_avg(agg7#39)] -Aggregate Attributes [14]: [sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119] -Results [17]: [i_item_id#105, ca_country#101, ca_state#100, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133] +Aggregate Attributes [14]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97] +Results [17]: [i_item_id#31, ca_country#27, ca_state#26, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111] (67) Exchange -Input [17]: [i_item_id#105, ca_country#101, ca_state#100, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133] -Arguments: hashpartitioning(i_item_id#105, ca_country#101, ca_state#100, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [17]: [i_item_id#31, ca_country#27, ca_state#26, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111] +Arguments: hashpartitioning(i_item_id#31, ca_country#27, ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#112] (68) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#105, ca_country#101, ca_state#100, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133] -Keys [3]: [i_item_id#105, ca_country#101, ca_state#100] +Input [17]: [i_item_id#31, ca_country#27, ca_state#26, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111] +Keys [3]: [i_item_id#31, ca_country#27, ca_state#26] Functions [7]: [avg(agg1#33), avg(agg2#34), avg(agg3#35), avg(agg4#36), avg(agg5#37), avg(agg6#38), avg(agg7#39)] -Aggregate Attributes [7]: [avg(agg1#33)#135, avg(agg2#34)#136, avg(agg3#35)#137, avg(agg4#36)#138, avg(agg5#37)#139, avg(agg6#38)#140, avg(agg7#39)#141] -Results [11]: [i_item_id#105, ca_country#101, ca_state#100, null AS county#142, avg(agg1#33)#135 AS agg1#143, avg(agg2#34)#136 AS agg2#144, avg(agg3#35)#137 AS agg3#145, avg(agg4#36)#138 AS agg4#146, avg(agg5#37)#139 AS agg5#147, avg(agg6#38)#140 AS agg6#148, avg(agg7#39)#141 AS agg7#149] +Aggregate Attributes [7]: [avg(agg1#33)#113, avg(agg2#34)#114, avg(agg3#35)#115, avg(agg4#36)#116, avg(agg5#37)#117, avg(agg6#38)#118, avg(agg7#39)#119] +Results [11]: [i_item_id#31, ca_country#27, ca_state#26, null AS county#120, avg(agg1#33)#113 AS agg1#121, avg(agg2#34)#114 AS agg2#122, avg(agg3#35)#115 AS agg3#123, avg(agg4#36)#116 AS agg4#124, avg(agg5#37)#117 AS agg5#125, avg(agg6#38)#118 AS agg6#126, avg(agg7#39)#119 AS agg7#127] (69) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#158), dynamicpruningexpression(cs_sold_date_sk#158 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (71) Filter [codegen id : 23] -Input [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] -Condition : ((isnotnull(cs_bill_cdemo_sk#151) AND isnotnull(cs_bill_customer_sk#150)) AND isnotnull(cs_item_sk#152)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (72) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#159, cd_dep_count#160] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (73) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#151] -Right keys [1]: [cd_demo_sk#159] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (74) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#150, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160] -Input [11]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_demo_sk#159, cd_dep_count#160] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#161, c_current_cdemo_sk#162, c_current_addr_sk#163, c_birth_year#164] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (76) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#150] -Right keys [1]: [c_customer_sk#161] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (77) Project [codegen id : 23] -Output [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_current_cdemo_sk#162, c_current_addr_sk#163, c_birth_year#164] -Input [13]: [cs_bill_customer_sk#150, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_customer_sk#161, c_current_cdemo_sk#162, c_current_addr_sk#163, c_birth_year#164] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (78) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#165] +Output [1]: [cd_demo_sk#22] (79) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#162] -Right keys [1]: [cd_demo_sk#165] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#22] Join condition: None (80) Project [codegen id : 23] -Output [10]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_current_addr_sk#163, c_birth_year#164] -Input [12]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_current_cdemo_sk#162, c_current_addr_sk#163, c_birth_year#164, cd_demo_sk#165] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20, cd_demo_sk#22] (81) Scan parquet default.customer_address -Output [3]: [ca_address_sk#166, ca_state#167, ca_country#168] +Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct (82) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#166, ca_state#167, ca_country#168] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (83) Filter [codegen id : 20] -Input [3]: [ca_address_sk#166, ca_state#167, ca_country#168] -Condition : (ca_state#167 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#166)) +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Condition : (ca_state#26 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#24)) (84) Project [codegen id : 20] -Output [2]: [ca_address_sk#166, ca_country#168] -Input [3]: [ca_address_sk#166, ca_state#167, ca_country#168] +Output [2]: [ca_address_sk#24, ca_country#27] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (85) BroadcastExchange -Input [2]: [ca_address_sk#166, ca_country#168] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#169] +Input [2]: [ca_address_sk#24, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#128] (86) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#163] -Right keys [1]: [ca_address_sk#166] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (87) Project [codegen id : 23] -Output [10]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_birth_year#164, ca_country#168] -Input [12]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_current_addr_sk#163, c_birth_year#164, ca_address_sk#166, ca_country#168] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20, ca_country#27] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20, ca_address_sk#24, ca_country#27] (88) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#170] +Output [1]: [d_date_sk#29] (89) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#158] -Right keys [1]: [d_date_sk#170] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (90) Project [codegen id : 23] -Output [9]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cd_dep_count#160, c_birth_year#164, ca_country#168] -Input [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#160, c_birth_year#164, ca_country#168, d_date_sk#170] +Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20, ca_country#27] +Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20, ca_country#27, d_date_sk#29] (91) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#171, i_item_id#172] +Output [2]: [i_item_sk#30, i_item_id#31] (92) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#152] -Right keys [1]: [i_item_sk#171] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#30] Join condition: None (93) Project [codegen id : 23] -Output [9]: [i_item_id#172, ca_country#168, cast(cs_quantity#153 as decimal(12,2)) AS agg1#33, cast(cs_list_price#154 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#156 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#155 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#157 as decimal(12,2)) AS agg5#37, cast(c_birth_year#164 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#160 as decimal(12,2)) AS agg7#39] -Input [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cd_dep_count#160, c_birth_year#164, ca_country#168, i_item_sk#171, i_item_id#172] +Output [9]: [i_item_id#31, ca_country#27, cast(cs_quantity#4 as decimal(12,2)) AS agg1#33, cast(cs_list_price#5 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#37, cast(c_birth_year#20 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#39] +Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20, ca_country#27, i_item_sk#30, i_item_id#31] (94) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#172, ca_country#168, agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] -Keys [2]: [i_item_id#172, ca_country#168] +Input [9]: [i_item_id#31, ca_country#27, agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] +Keys [2]: [i_item_id#31, ca_country#27] Functions [7]: [partial_avg(agg1#33), partial_avg(agg2#34), partial_avg(agg3#35), partial_avg(agg4#36), partial_avg(agg5#37), partial_avg(agg6#38), partial_avg(agg7#39)] -Aggregate Attributes [14]: [sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184, sum#185, count#186] -Results [16]: [i_item_id#172, ca_country#168, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200] +Aggregate Attributes [14]: [sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142] +Results [16]: [i_item_id#31, ca_country#27, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156] (95) Exchange -Input [16]: [i_item_id#172, ca_country#168, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200] -Arguments: hashpartitioning(i_item_id#172, ca_country#168, 5), ENSURE_REQUIREMENTS, [id=#201] +Input [16]: [i_item_id#31, ca_country#27, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156] +Arguments: hashpartitioning(i_item_id#31, ca_country#27, 5), ENSURE_REQUIREMENTS, [id=#157] (96) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#172, ca_country#168, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200] -Keys [2]: [i_item_id#172, ca_country#168] +Input [16]: [i_item_id#31, ca_country#27, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156] +Keys [2]: [i_item_id#31, ca_country#27] Functions [7]: [avg(agg1#33), avg(agg2#34), avg(agg3#35), avg(agg4#36), avg(agg5#37), avg(agg6#38), avg(agg7#39)] -Aggregate Attributes [7]: [avg(agg1#33)#202, avg(agg2#34)#203, avg(agg3#35)#204, avg(agg4#36)#205, avg(agg5#37)#206, avg(agg6#38)#207, avg(agg7#39)#208] -Results [11]: [i_item_id#172, ca_country#168, null AS ca_state#209, null AS county#210, avg(agg1#33)#202 AS agg1#211, avg(agg2#34)#203 AS agg2#212, avg(agg3#35)#204 AS agg3#213, avg(agg4#36)#205 AS agg4#214, avg(agg5#37)#206 AS agg5#215, avg(agg6#38)#207 AS agg6#216, avg(agg7#39)#208 AS agg7#217] +Aggregate Attributes [7]: [avg(agg1#33)#158, avg(agg2#34)#159, avg(agg3#35)#160, avg(agg4#36)#161, avg(agg5#37)#162, avg(agg6#38)#163, avg(agg7#39)#164] +Results [11]: [i_item_id#31, ca_country#27, null AS ca_state#165, null AS county#166, avg(agg1#33)#158 AS agg1#167, avg(agg2#34)#159 AS agg2#168, avg(agg3#35)#160 AS agg3#169, avg(agg4#36)#161 AS agg4#170, avg(agg5#37)#162 AS agg5#171, avg(agg6#38)#163 AS agg6#172, avg(agg7#39)#164 AS agg7#173] (97) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#218, cs_bill_cdemo_sk#219, cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#226), dynamicpruningexpression(cs_sold_date_sk#226 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#218, cs_bill_cdemo_sk#219, cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (99) Filter [codegen id : 31] -Input [9]: [cs_bill_customer_sk#218, cs_bill_cdemo_sk#219, cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226] -Condition : ((isnotnull(cs_bill_cdemo_sk#219) AND isnotnull(cs_bill_customer_sk#218)) AND isnotnull(cs_item_sk#220)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (100) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#227, cd_dep_count#228] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (101) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#219] -Right keys [1]: [cd_demo_sk#227] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (102) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#218, cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228] -Input [11]: [cs_bill_customer_sk#218, cs_bill_cdemo_sk#219, cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_demo_sk#227, cd_dep_count#228] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (103) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#229, c_current_cdemo_sk#230, c_current_addr_sk#231, c_birth_year#232] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (104) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#218] -Right keys [1]: [c_customer_sk#229] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (105) Project [codegen id : 31] -Output [11]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_current_cdemo_sk#230, c_current_addr_sk#231, c_birth_year#232] -Input [13]: [cs_bill_customer_sk#218, cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_customer_sk#229, c_current_cdemo_sk#230, c_current_addr_sk#231, c_birth_year#232] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (106) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#233] +Output [1]: [cd_demo_sk#22] (107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#230] -Right keys [1]: [cd_demo_sk#233] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#22] Join condition: None (108) Project [codegen id : 31] -Output [10]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_current_addr_sk#231, c_birth_year#232] -Input [12]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_current_cdemo_sk#230, c_current_addr_sk#231, c_birth_year#232, cd_demo_sk#233] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20, cd_demo_sk#22] (109) Scan parquet default.customer_address -Output [2]: [ca_address_sk#234, ca_state#235] +Output [2]: [ca_address_sk#24, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct (110) ColumnarToRow [codegen id : 28] -Input [2]: [ca_address_sk#234, ca_state#235] +Input [2]: [ca_address_sk#24, ca_state#26] (111) Filter [codegen id : 28] -Input [2]: [ca_address_sk#234, ca_state#235] -Condition : (ca_state#235 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#234)) +Input [2]: [ca_address_sk#24, ca_state#26] +Condition : (ca_state#26 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#24)) (112) Project [codegen id : 28] -Output [1]: [ca_address_sk#234] -Input [2]: [ca_address_sk#234, ca_state#235] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#26] (113) BroadcastExchange -Input [1]: [ca_address_sk#234] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#236] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#174] (114) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#231] -Right keys [1]: [ca_address_sk#234] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (115) Project [codegen id : 31] -Output [9]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_birth_year#232] -Input [11]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_current_addr_sk#231, c_birth_year#232, ca_address_sk#234] +Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20] +Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20, ca_address_sk#24] (116) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#237] +Output [1]: [d_date_sk#29] (117) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#226] -Right keys [1]: [d_date_sk#237] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (118) Project [codegen id : 31] -Output [8]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cd_dep_count#228, c_birth_year#232] -Input [10]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cs_sold_date_sk#226, cd_dep_count#228, c_birth_year#232, d_date_sk#237] +Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20] +Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20, d_date_sk#29] (119) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#238, i_item_id#239] +Output [2]: [i_item_sk#30, i_item_id#31] (120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#220] -Right keys [1]: [i_item_sk#238] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#30] Join condition: None (121) Project [codegen id : 31] -Output [8]: [i_item_id#239, cast(cs_quantity#221 as decimal(12,2)) AS agg1#33, cast(cs_list_price#222 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#224 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#223 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#225 as decimal(12,2)) AS agg5#37, cast(c_birth_year#232 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#228 as decimal(12,2)) AS agg7#39] -Input [10]: [cs_item_sk#220, cs_quantity#221, cs_list_price#222, cs_sales_price#223, cs_coupon_amt#224, cs_net_profit#225, cd_dep_count#228, c_birth_year#232, i_item_sk#238, i_item_id#239] +Output [8]: [i_item_id#31, cast(cs_quantity#4 as decimal(12,2)) AS agg1#33, cast(cs_list_price#5 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#37, cast(c_birth_year#20 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#39] +Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20, i_item_sk#30, i_item_id#31] (122) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#239, agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] -Keys [1]: [i_item_id#239] +Input [8]: [i_item_id#31, agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] +Keys [1]: [i_item_id#31] Functions [7]: [partial_avg(agg1#33), partial_avg(agg2#34), partial_avg(agg3#35), partial_avg(agg4#36), partial_avg(agg5#37), partial_avg(agg6#38), partial_avg(agg7#39)] -Aggregate Attributes [14]: [sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253] -Results [15]: [i_item_id#239, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267] +Aggregate Attributes [14]: [sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188] +Results [15]: [i_item_id#31, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200, sum#201, count#202] (123) Exchange -Input [15]: [i_item_id#239, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267] -Arguments: hashpartitioning(i_item_id#239, 5), ENSURE_REQUIREMENTS, [id=#268] +Input [15]: [i_item_id#31, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200, sum#201, count#202] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#203] (124) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#239, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267] -Keys [1]: [i_item_id#239] +Input [15]: [i_item_id#31, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200, sum#201, count#202] +Keys [1]: [i_item_id#31] Functions [7]: [avg(agg1#33), avg(agg2#34), avg(agg3#35), avg(agg4#36), avg(agg5#37), avg(agg6#38), avg(agg7#39)] -Aggregate Attributes [7]: [avg(agg1#33)#269, avg(agg2#34)#270, avg(agg3#35)#271, avg(agg4#36)#272, avg(agg5#37)#273, avg(agg6#38)#274, avg(agg7#39)#275] -Results [11]: [i_item_id#239, null AS ca_country#276, null AS ca_state#277, null AS county#278, avg(agg1#33)#269 AS agg1#279, avg(agg2#34)#270 AS agg2#280, avg(agg3#35)#271 AS agg3#281, avg(agg4#36)#272 AS agg4#282, avg(agg5#37)#273 AS agg5#283, avg(agg6#38)#274 AS agg6#284, avg(agg7#39)#275 AS agg7#285] +Aggregate Attributes [7]: [avg(agg1#33)#204, avg(agg2#34)#205, avg(agg3#35)#206, avg(agg4#36)#207, avg(agg5#37)#208, avg(agg6#38)#209, avg(agg7#39)#210] +Results [11]: [i_item_id#31, null AS ca_country#211, null AS ca_state#212, null AS county#213, avg(agg1#33)#204 AS agg1#214, avg(agg2#34)#205 AS agg2#215, avg(agg3#35)#206 AS agg3#216, avg(agg4#36)#207 AS agg4#217, avg(agg5#37)#208 AS agg5#218, avg(agg6#38)#209 AS agg6#219, avg(agg7#39)#210 AS agg7#220] (125) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#286, cs_bill_cdemo_sk#287, cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#294), dynamicpruningexpression(cs_sold_date_sk#294 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (126) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#286, cs_bill_cdemo_sk#287, cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (127) Filter [codegen id : 39] -Input [9]: [cs_bill_customer_sk#286, cs_bill_cdemo_sk#287, cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294] -Condition : ((isnotnull(cs_bill_cdemo_sk#287) AND isnotnull(cs_bill_customer_sk#286)) AND isnotnull(cs_item_sk#288)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (128) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#295, cd_dep_count#296] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (129) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#287] -Right keys [1]: [cd_demo_sk#295] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (130) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#286, cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296] -Input [11]: [cs_bill_customer_sk#286, cs_bill_cdemo_sk#287, cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_demo_sk#295, cd_dep_count#296] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (131) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#297, c_current_cdemo_sk#298, c_current_addr_sk#299, c_birth_year#300] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (132) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#286] -Right keys [1]: [c_customer_sk#297] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (133) Project [codegen id : 39] -Output [11]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_current_cdemo_sk#298, c_current_addr_sk#299, c_birth_year#300] -Input [13]: [cs_bill_customer_sk#286, cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_customer_sk#297, c_current_cdemo_sk#298, c_current_addr_sk#299, c_birth_year#300] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (134) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#301] +Output [1]: [cd_demo_sk#22] (135) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#298] -Right keys [1]: [cd_demo_sk#301] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#22] Join condition: None (136) Project [codegen id : 39] -Output [10]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_current_addr_sk#299, c_birth_year#300] -Input [12]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_current_cdemo_sk#298, c_current_addr_sk#299, c_birth_year#300, cd_demo_sk#301] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20] +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20, cd_demo_sk#22] (137) ReusedExchange [Reuses operator id: 113] -Output [1]: [ca_address_sk#302] +Output [1]: [ca_address_sk#24] (138) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#299] -Right keys [1]: [ca_address_sk#302] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (139) Project [codegen id : 39] -Output [9]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_birth_year#300] -Input [11]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_current_addr_sk#299, c_birth_year#300, ca_address_sk#302] +Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20] +Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#18, c_birth_year#20, ca_address_sk#24] (140) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#303] +Output [1]: [d_date_sk#29] (141) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#294] -Right keys [1]: [d_date_sk#303] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (142) Project [codegen id : 39] -Output [8]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cd_dep_count#296, c_birth_year#300] -Input [10]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cs_sold_date_sk#294, cd_dep_count#296, c_birth_year#300, d_date_sk#303] +Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20] +Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#20, d_date_sk#29] (143) Scan parquet default.item -Output [1]: [i_item_sk#304] +Output [1]: [i_item_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#304] +Input [1]: [i_item_sk#30] (145) Filter [codegen id : 38] -Input [1]: [i_item_sk#304] -Condition : isnotnull(i_item_sk#304) +Input [1]: [i_item_sk#30] +Condition : isnotnull(i_item_sk#30) (146) BroadcastExchange -Input [1]: [i_item_sk#304] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#305] +Input [1]: [i_item_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#221] (147) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#288] -Right keys [1]: [i_item_sk#304] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#30] Join condition: None (148) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#289 as decimal(12,2)) AS agg1#33, cast(cs_list_price#290 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#292 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#291 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#293 as decimal(12,2)) AS agg5#37, cast(c_birth_year#300 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#296 as decimal(12,2)) AS agg7#39] -Input [9]: [cs_item_sk#288, cs_quantity#289, cs_list_price#290, cs_sales_price#291, cs_coupon_amt#292, cs_net_profit#293, cd_dep_count#296, c_birth_year#300, i_item_sk#304] +Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#33, cast(cs_list_price#5 as decimal(12,2)) AS agg2#34, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#35, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#36, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#37, cast(c_birth_year#20 as decimal(12,2)) AS agg6#38, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#39] +Input [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#20, i_item_sk#30] (149) HashAggregate [codegen id : 39] Input [7]: [agg1#33, agg2#34, agg3#35, agg4#36, agg5#37, agg6#38, agg7#39] Keys: [] Functions [7]: [partial_avg(agg1#33), partial_avg(agg2#34), partial_avg(agg3#35), partial_avg(agg4#36), partial_avg(agg5#37), partial_avg(agg6#38), partial_avg(agg7#39)] -Aggregate Attributes [14]: [sum#306, count#307, sum#308, count#309, sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319] -Results [14]: [sum#320, count#321, sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333] +Aggregate Attributes [14]: [sum#222, count#223, sum#224, count#225, sum#226, count#227, sum#228, count#229, sum#230, count#231, sum#232, count#233, sum#234, count#235] +Results [14]: [sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249] (150) Exchange -Input [14]: [sum#320, count#321, sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#334] +Input [14]: [sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#250] (151) HashAggregate [codegen id : 40] -Input [14]: [sum#320, count#321, sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333] +Input [14]: [sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249] Keys: [] Functions [7]: [avg(agg1#33), avg(agg2#34), avg(agg3#35), avg(agg4#36), avg(agg5#37), avg(agg6#38), avg(agg7#39)] -Aggregate Attributes [7]: [avg(agg1#33)#335, avg(agg2#34)#336, avg(agg3#35)#337, avg(agg4#36)#338, avg(agg5#37)#339, avg(agg6#38)#340, avg(agg7#39)#341] -Results [11]: [null AS i_item_id#342, null AS ca_country#343, null AS ca_state#344, null AS county#345, avg(agg1#33)#335 AS agg1#346, avg(agg2#34)#336 AS agg2#347, avg(agg3#35)#337 AS agg3#348, avg(agg4#36)#338 AS agg4#349, avg(agg5#37)#339 AS agg5#350, avg(agg6#38)#340 AS agg6#351, avg(agg7#39)#341 AS agg7#352] +Aggregate Attributes [7]: [avg(agg1#33)#251, avg(agg2#34)#252, avg(agg3#35)#253, avg(agg4#36)#254, avg(agg5#37)#255, avg(agg6#38)#256, avg(agg7#39)#257] +Results [11]: [null AS i_item_id#258, null AS ca_country#259, null AS ca_state#260, null AS county#261, avg(agg1#33)#251 AS agg1#262, avg(agg2#34)#252 AS agg2#263, avg(agg3#35)#253 AS agg3#264, avg(agg4#36)#254 AS agg4#265, avg(agg5#37)#255 AS agg5#266, avg(agg6#38)#256 AS agg6#267, avg(agg7#39)#257 AS agg7#268] (152) Union @@ -847,33 +847,33 @@ BroadcastExchange (158) (154) Scan parquet default.date_dim -Output [2]: [d_date_sk#29, d_year#353] +Output [2]: [d_date_sk#29, d_year#269] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (155) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_year#353] +Input [2]: [d_date_sk#29, d_year#269] (156) Filter [codegen id : 1] -Input [2]: [d_date_sk#29, d_year#353] -Condition : ((isnotnull(d_year#353) AND (d_year#353 = 2001)) AND isnotnull(d_date_sk#29)) +Input [2]: [d_date_sk#29, d_year#269] +Condition : ((isnotnull(d_year#269) AND (d_year#269 = 2001)) AND isnotnull(d_date_sk#29)) (157) Project [codegen id : 1] Output [1]: [d_date_sk#29] -Input [2]: [d_date_sk#29, d_year#353] +Input [2]: [d_date_sk#29, d_year#269] (158) BroadcastExchange Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#354] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#270] -Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#158 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#226 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#294 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt index c72615461cff4..e7072101f8f23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt @@ -175,116 +175,116 @@ Aggregate Attributes [1]: [avg(qoh#22)#27] Results [5]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, avg(qoh#22)#27 AS qoh#28] (27) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Output [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] (28) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] -Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] -Functions [1]: [avg(inv_quantity_on_hand#35)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#36] -Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#36 AS qoh#22] +Input [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] +Keys [4]: [i_product_name#14, i_brand#11, i_class#12, i_category#13] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [4]: [i_product_name#14, i_brand#11, i_class#12, avg(inv_quantity_on_hand#3)#21 AS qoh#22] (29) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#22] -Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Input [4]: [i_product_name#14, i_brand#11, i_class#12, qoh#22] +Keys [3]: [i_product_name#14, i_brand#11, i_class#12] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#37, count#38] -Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Aggregate Attributes [2]: [sum#29, count#30] +Results [5]: [i_product_name#14, i_brand#11, i_class#12, sum#31, count#32] (30) Exchange -Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] -Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [i_product_name#14, i_brand#11, i_class#12, sum#31, count#32] +Arguments: hashpartitioning(i_product_name#14, i_brand#11, i_class#12, 5), ENSURE_REQUIREMENTS, [id=#33] (31) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] -Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Input [5]: [i_product_name#14, i_brand#11, i_class#12, sum#31, count#32] +Keys [3]: [i_product_name#14, i_brand#11, i_class#12] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#42] -Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#43, avg(qoh#22)#42 AS qoh#44] +Aggregate Attributes [1]: [avg(qoh#22)#34] +Results [5]: [i_product_name#14, i_brand#11, i_class#12, null AS i_category#35, avg(qoh#22)#34 AS qoh#36] (32) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] +Output [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] (33) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] -Keys [4]: [i_product_name#45, i_brand#46, i_class#47, i_category#48] -Functions [1]: [avg(inv_quantity_on_hand#51)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#51)#52] -Results [3]: [i_product_name#45, i_brand#46, avg(inv_quantity_on_hand#51)#52 AS qoh#22] +Input [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] +Keys [4]: [i_product_name#14, i_brand#11, i_class#12, i_category#13] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [3]: [i_product_name#14, i_brand#11, avg(inv_quantity_on_hand#3)#21 AS qoh#22] (34) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#45, i_brand#46, qoh#22] -Keys [2]: [i_product_name#45, i_brand#46] +Input [3]: [i_product_name#14, i_brand#11, qoh#22] +Keys [2]: [i_product_name#14, i_brand#11] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#53, count#54] -Results [4]: [i_product_name#45, i_brand#46, sum#55, count#56] +Aggregate Attributes [2]: [sum#37, count#38] +Results [4]: [i_product_name#14, i_brand#11, sum#39, count#40] (35) Exchange -Input [4]: [i_product_name#45, i_brand#46, sum#55, count#56] -Arguments: hashpartitioning(i_product_name#45, i_brand#46, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [i_product_name#14, i_brand#11, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#14, i_brand#11, 5), ENSURE_REQUIREMENTS, [id=#41] (36) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#45, i_brand#46, sum#55, count#56] -Keys [2]: [i_product_name#45, i_brand#46] +Input [4]: [i_product_name#14, i_brand#11, sum#39, count#40] +Keys [2]: [i_product_name#14, i_brand#11] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#58] -Results [5]: [i_product_name#45, i_brand#46, null AS i_class#59, null AS i_category#60, avg(qoh#22)#58 AS qoh#61] +Aggregate Attributes [1]: [avg(qoh#22)#42] +Results [5]: [i_product_name#14, i_brand#11, null AS i_class#43, null AS i_category#44, avg(qoh#22)#42 AS qoh#45] (37) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#62, i_brand#63, i_class#64, i_category#65, sum#66, count#67] +Output [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] (38) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#62, i_brand#63, i_class#64, i_category#65, sum#66, count#67] -Keys [4]: [i_product_name#62, i_brand#63, i_class#64, i_category#65] -Functions [1]: [avg(inv_quantity_on_hand#68)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#68)#69] -Results [2]: [i_product_name#62, avg(inv_quantity_on_hand#68)#69 AS qoh#22] +Input [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] +Keys [4]: [i_product_name#14, i_brand#11, i_class#12, i_category#13] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [2]: [i_product_name#14, avg(inv_quantity_on_hand#3)#21 AS qoh#22] (39) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#62, qoh#22] -Keys [1]: [i_product_name#62] +Input [2]: [i_product_name#14, qoh#22] +Keys [1]: [i_product_name#14] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [3]: [i_product_name#62, sum#72, count#73] +Aggregate Attributes [2]: [sum#46, count#47] +Results [3]: [i_product_name#14, sum#48, count#49] (40) Exchange -Input [3]: [i_product_name#62, sum#72, count#73] -Arguments: hashpartitioning(i_product_name#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [i_product_name#14, sum#48, count#49] +Arguments: hashpartitioning(i_product_name#14, 5), ENSURE_REQUIREMENTS, [id=#50] (41) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#62, sum#72, count#73] -Keys [1]: [i_product_name#62] +Input [3]: [i_product_name#14, sum#48, count#49] +Keys [1]: [i_product_name#14] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#75] -Results [5]: [i_product_name#62, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#22)#75 AS qoh#79] +Aggregate Attributes [1]: [avg(qoh#22)#51] +Results [5]: [i_product_name#14, null AS i_brand#52, null AS i_class#53, null AS i_category#54, avg(qoh#22)#51 AS qoh#55] (42) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#80, i_brand#81, i_class#82, i_category#83, sum#84, count#85] +Output [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] (43) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#80, i_brand#81, i_class#82, i_category#83, sum#84, count#85] -Keys [4]: [i_product_name#80, i_brand#81, i_class#82, i_category#83] -Functions [1]: [avg(inv_quantity_on_hand#86)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#86)#87] -Results [1]: [avg(inv_quantity_on_hand#86)#87 AS qoh#22] +Input [6]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, sum#18, count#19] +Keys [4]: [i_product_name#14, i_brand#11, i_class#12, i_category#13] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [1]: [avg(inv_quantity_on_hand#3)#21 AS qoh#22] (44) HashAggregate [codegen id : 43] Input [1]: [qoh#22] Keys: [] Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#88, count#89] -Results [2]: [sum#90, count#91] +Aggregate Attributes [2]: [sum#56, count#57] +Results [2]: [sum#58, count#59] (45) Exchange -Input [2]: [sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#92] +Input [2]: [sum#58, count#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] (46) HashAggregate [codegen id : 44] -Input [2]: [sum#90, count#91] +Input [2]: [sum#58, count#59] Keys: [] Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#93] -Results [5]: [null AS i_product_name#94, null AS i_brand#95, null AS i_class#96, null AS i_category#97, avg(qoh#22)#93 AS qoh#98] +Aggregate Attributes [1]: [avg(qoh#22)#61] +Results [5]: [null AS i_product_name#62, null AS i_brand#63, null AS i_class#64, null AS i_category#65, avg(qoh#22)#61 AS qoh#66] (47) Union @@ -303,25 +303,25 @@ BroadcastExchange (53) (49) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#99] +Output [2]: [d_date_sk#8, d_month_seq#67] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#99] +Input [2]: [d_date_sk#8, d_month_seq#67] (51) Filter [codegen id : 1] -Input [2]: [d_date_sk#8, d_month_seq#99] -Condition : (((isnotnull(d_month_seq#99) AND (d_month_seq#99 >= 1212)) AND (d_month_seq#99 <= 1223)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#8, d_month_seq#67] +Condition : (((isnotnull(d_month_seq#67) AND (d_month_seq#67 >= 1212)) AND (d_month_seq#67 <= 1223)) AND isnotnull(d_date_sk#8)) (52) Project [codegen id : 1] Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_month_seq#99] +Input [2]: [d_date_sk#8, d_month_seq#67] (53) BroadcastExchange Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index b2786e636a69b..c5bef0d13db91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -160,116 +160,116 @@ Aggregate Attributes [1]: [avg(qoh#21)#26] Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#21)#26 AS qoh#27] (24) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#28, i_brand#29, i_class#30, i_category#31, sum#32, count#33] +Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] (25) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#28, i_brand#29, i_class#30, i_category#31, sum#32, count#33] -Keys [4]: [i_product_name#28, i_brand#29, i_class#30, i_category#31] -Functions [1]: [avg(inv_quantity_on_hand#34)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#34)#35] -Results [4]: [i_product_name#28, i_brand#29, i_class#30, avg(inv_quantity_on_hand#34)#35 AS qoh#21] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#20] +Results [4]: [i_product_name#11, i_brand#8, i_class#9, avg(inv_quantity_on_hand#3)#20 AS qoh#21] (26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#28, i_brand#29, i_class#30, qoh#21] -Keys [3]: [i_product_name#28, i_brand#29, i_class#30] +Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#21] +Keys [3]: [i_product_name#11, i_brand#8, i_class#9] Functions [1]: [partial_avg(qoh#21)] -Aggregate Attributes [2]: [sum#36, count#37] -Results [5]: [i_product_name#28, i_brand#29, i_class#30, sum#38, count#39] +Aggregate Attributes [2]: [sum#28, count#29] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, sum#30, count#31] (27) Exchange -Input [5]: [i_product_name#28, i_brand#29, i_class#30, sum#38, count#39] -Arguments: hashpartitioning(i_product_name#28, i_brand#29, i_class#30, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#30, count#31] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, [id=#32] (28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#28, i_brand#29, i_class#30, sum#38, count#39] -Keys [3]: [i_product_name#28, i_brand#29, i_class#30] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#30, count#31] +Keys [3]: [i_product_name#11, i_brand#8, i_class#9] Functions [1]: [avg(qoh#21)] -Aggregate Attributes [1]: [avg(qoh#21)#41] -Results [5]: [i_product_name#28, i_brand#29, i_class#30, null AS i_category#42, avg(qoh#21)#41 AS qoh#43] +Aggregate Attributes [1]: [avg(qoh#21)#33] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, null AS i_category#34, avg(qoh#21)#33 AS qoh#35] (29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] (30) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] -Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] -Functions [1]: [avg(inv_quantity_on_hand#50)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#51] -Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#51 AS qoh#21] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#20] +Results [3]: [i_product_name#11, i_brand#8, avg(inv_quantity_on_hand#3)#20 AS qoh#21] (31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#44, i_brand#45, qoh#21] -Keys [2]: [i_product_name#44, i_brand#45] +Input [3]: [i_product_name#11, i_brand#8, qoh#21] +Keys [2]: [i_product_name#11, i_brand#8] Functions [1]: [partial_avg(qoh#21)] -Aggregate Attributes [2]: [sum#52, count#53] -Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Aggregate Attributes [2]: [sum#36, count#37] +Results [4]: [i_product_name#11, i_brand#8, sum#38, count#39] (32) Exchange -Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] -Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [i_product_name#11, i_brand#8, sum#38, count#39] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, [id=#40] (33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] -Keys [2]: [i_product_name#44, i_brand#45] +Input [4]: [i_product_name#11, i_brand#8, sum#38, count#39] +Keys [2]: [i_product_name#11, i_brand#8] Functions [1]: [avg(qoh#21)] -Aggregate Attributes [1]: [avg(qoh#21)#57] -Results [5]: [i_product_name#44, i_brand#45, null AS i_class#58, null AS i_category#59, avg(qoh#21)#57 AS qoh#60] +Aggregate Attributes [1]: [avg(qoh#21)#41] +Results [5]: [i_product_name#11, i_brand#8, null AS i_class#42, null AS i_category#43, avg(qoh#21)#41 AS qoh#44] (34) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] +Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] (35) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] -Keys [4]: [i_product_name#61, i_brand#62, i_class#63, i_category#64] -Functions [1]: [avg(inv_quantity_on_hand#67)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#67)#68] -Results [2]: [i_product_name#61, avg(inv_quantity_on_hand#67)#68 AS qoh#21] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#20] +Results [2]: [i_product_name#11, avg(inv_quantity_on_hand#3)#20 AS qoh#21] (36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#61, qoh#21] -Keys [1]: [i_product_name#61] +Input [2]: [i_product_name#11, qoh#21] +Keys [1]: [i_product_name#11] Functions [1]: [partial_avg(qoh#21)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [3]: [i_product_name#61, sum#71, count#72] +Aggregate Attributes [2]: [sum#45, count#46] +Results [3]: [i_product_name#11, sum#47, count#48] (37) Exchange -Input [3]: [i_product_name#61, sum#71, count#72] -Arguments: hashpartitioning(i_product_name#61, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [3]: [i_product_name#11, sum#47, count#48] +Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, [id=#49] (38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#61, sum#71, count#72] -Keys [1]: [i_product_name#61] +Input [3]: [i_product_name#11, sum#47, count#48] +Keys [1]: [i_product_name#11] Functions [1]: [avg(qoh#21)] -Aggregate Attributes [1]: [avg(qoh#21)#74] -Results [5]: [i_product_name#61, null AS i_brand#75, null AS i_class#76, null AS i_category#77, avg(qoh#21)#74 AS qoh#78] +Aggregate Attributes [1]: [avg(qoh#21)#50] +Results [5]: [i_product_name#11, null AS i_brand#51, null AS i_class#52, null AS i_category#53, avg(qoh#21)#50 AS qoh#54] (39) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#79, i_brand#80, i_class#81, i_category#82, sum#83, count#84] +Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] (40) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#79, i_brand#80, i_class#81, i_category#82, sum#83, count#84] -Keys [4]: [i_product_name#79, i_brand#80, i_class#81, i_category#82] -Functions [1]: [avg(inv_quantity_on_hand#85)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#85)#86] -Results [1]: [avg(inv_quantity_on_hand#85)#86 AS qoh#21] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#17, count#18] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#20] +Results [1]: [avg(inv_quantity_on_hand#3)#20 AS qoh#21] (41) HashAggregate [codegen id : 28] Input [1]: [qoh#21] Keys: [] Functions [1]: [partial_avg(qoh#21)] -Aggregate Attributes [2]: [sum#87, count#88] -Results [2]: [sum#89, count#90] +Aggregate Attributes [2]: [sum#55, count#56] +Results [2]: [sum#57, count#58] (42) Exchange -Input [2]: [sum#89, count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +Input [2]: [sum#57, count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] (43) HashAggregate [codegen id : 29] -Input [2]: [sum#89, count#90] +Input [2]: [sum#57, count#58] Keys: [] Functions [1]: [avg(qoh#21)] -Aggregate Attributes [1]: [avg(qoh#21)#92] -Results [5]: [null AS i_product_name#93, null AS i_brand#94, null AS i_class#95, null AS i_category#96, avg(qoh#21)#92 AS qoh#97] +Aggregate Attributes [1]: [avg(qoh#21)#60] +Results [5]: [null AS i_product_name#61, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#21)#60 AS qoh#65] (44) Union @@ -288,25 +288,25 @@ BroadcastExchange (50) (46) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#98] +Output [2]: [d_date_sk#6, d_month_seq#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#98] +Input [2]: [d_date_sk#6, d_month_seq#66] (48) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#98] -Condition : (((isnotnull(d_month_seq#98) AND (d_month_seq#98 >= 1212)) AND (d_month_seq#98 <= 1223)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_month_seq#66] +Condition : (((isnotnull(d_month_seq#66) AND (d_month_seq#66 >= 1212)) AND (d_month_seq#66 <= 1223)) AND isnotnull(d_date_sk#6)) (49) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#98] +Input [2]: [d_date_sk#6, d_month_seq#66] (50) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#99] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#67] 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 c8cd870c62c8a..c08379b07b397 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 @@ -304,132 +304,132 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (49) ReusedExchange [Reuses operator id: 17] -Output [7]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57] +Output [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15] (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +Output [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] 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 -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] (52) Filter -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] -Condition : (((isnotnull(ss_ticket_number#61) AND isnotnull(ss_item_sk#58)) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_customer_sk#59)) +Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : (((isnotnull(ss_ticket_number#21) AND isnotnull(ss_item_sk#18)) AND isnotnull(ss_store_sk#20)) AND isnotnull(ss_customer_sk#19)) (53) Project -Output [5]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] -Input [6]: [ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62, ss_sold_date_sk#63] +Output [5]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22] +Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] (54) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#51, c_customer_sk#55] -Right keys [2]: [ss_store_sk#60, ss_customer_sk#59] +Left keys [2]: [s_store_sk#1, c_customer_sk#12] +Right keys [2]: [ss_store_sk#20, ss_customer_sk#19] Join condition: None (55) Project [codegen id : 4] -Output [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Input [12]: [s_store_sk#51, s_store_name#52, s_state#53, ca_state#54, c_customer_sk#55, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_customer_sk#59, ss_store_sk#60, ss_ticket_number#61, ss_net_paid#62] +Output [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22] +Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk#12, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22] (56) Exchange -Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Arguments: hashpartitioning(ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22] +Arguments: hashpartitioning(ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#51] (57) Sort [codegen id : 5] -Input [8]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62] -Arguments: [ss_item_sk#58 ASC NULLS FIRST], false, 0 +Input [8]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22] +Arguments: [ss_item_sk#18 ASC NULLS FIRST], false, 0 (58) 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] +Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) 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] +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] (60) 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) +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Condition : isnotnull(i_item_sk#24) (61) Exchange -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: hashpartitioning(i_item_sk#65, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: hashpartitioning(i_item_sk#24, 5), ENSURE_REQUIREMENTS, [id=#52] (62) Sort [codegen id : 7] -Input [6]: [i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: [i_item_sk#65 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: [i_item_sk#24 ASC NULLS FIRST], false, 0 (63) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#58] -Right keys [1]: [i_item_sk#65] +Left keys [1]: [ss_item_sk#18] +Right keys [1]: [i_item_sk#24] Join condition: None (64) Project [codegen id : 8] -Output [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Input [14]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_item_sk#65, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] +Output [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Input [14]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] (65) Exchange -Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: hashpartitioning(ss_ticket_number#61, ss_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: hashpartitioning(ss_ticket_number#21, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#53] (66) Sort [codegen id : 9] -Input [13]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70] -Arguments: [ss_ticket_number#61 ASC NULLS FIRST, ss_item_sk#58 ASC NULLS FIRST], false, 0 +Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Arguments: [ss_ticket_number#21 ASC NULLS FIRST, ss_item_sk#18 ASC NULLS FIRST], false, 0 (67) ReusedExchange [Reuses operator id: 36] -Output [2]: [sr_item_sk#73, sr_ticket_number#74] +Output [2]: [sr_item_sk#32, sr_ticket_number#33] (68) Sort [codegen id : 11] -Input [2]: [sr_item_sk#73, sr_ticket_number#74] -Arguments: [sr_ticket_number#74 ASC NULLS FIRST, sr_item_sk#73 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#32, sr_ticket_number#33] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 12] -Left keys [2]: [ss_ticket_number#61, ss_item_sk#58] -Right keys [2]: [sr_ticket_number#74, sr_item_sk#73] +Left keys [2]: [ss_ticket_number#21, ss_item_sk#18] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (70) Project [codegen id : 12] -Output [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] -Input [15]: [s_store_name#52, s_state#53, ca_state#54, c_first_name#56, c_last_name#57, ss_item_sk#58, ss_ticket_number#61, ss_net_paid#62, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, sr_item_sk#73, sr_ticket_number#74] +Output [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] +Input [15]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, sr_item_sk#32, sr_ticket_number#33] (71) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#62, s_store_name#52, s_state#53, i_current_price#66, i_size#67, i_color#68, i_units#69, i_manager_id#70, c_first_name#56, c_last_name#57, ca_state#54] -Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, 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#62))] -Aggregate Attributes [1]: [sum#75] -Results [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] +Input [11]: [ss_net_paid#22, s_store_name#2, s_state#4, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29, c_first_name#14, c_last_name#15, ca_state#8] +Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#54] +Results [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55] (72) Exchange -Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] -Arguments: hashpartitioning(c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55] +Arguments: hashpartitioning(c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, 5), ENSURE_REQUIREMENTS, [id=#56] (73) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67, sum#76] -Keys [10]: [c_last_name#57, c_first_name#56, s_store_name#52, ca_state#54, s_state#53, i_color#68, i_current_price#66, i_manager_id#70, i_units#69, i_size#67] -Functions [1]: [sum(UnscaledValue(ss_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#62))#78] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#62))#78,17,2) AS netpaid#40] +Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#55] +Keys [10]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#39] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#39,17,2) AS netpaid#40] (74) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (75) Exchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#83] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (76) HashAggregate [codegen id : 14] -Input [2]: [sum#81, count#82] +Input [2]: [sum#59, count#60] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#84] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#84)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#85] +Aggregate Attributes [1]: [avg(netpaid#40)#62] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#62)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#63] 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 ef15af68693a3..d27e5af04e2dc 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 @@ -303,125 +303,125 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (49) ReusedExchange [Reuses operator id: 5] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] (50) 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 +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 (51) ReusedExchange [Reuses operator id: 11] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] (52) 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 +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: [sr_ticket_number#9 ASC NULLS FIRST, sr_item_sk#8 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 9] -Left keys [2]: [ss_ticket_number#54, ss_item_sk#51] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (54) 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#56, sr_ticket_number#57] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#8, sr_ticket_number#9] (55) ReusedExchange [Reuses operator id: 19] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] (56) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#53] -Right keys [1]: [s_store_sk#58] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (57) Project [codegen id : 9] -Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] (58) 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] +Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (60) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Condition : isnotnull(i_item_sk#18) (61) BroadcastExchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] (62) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#51] -Right keys [1]: [i_item_sk#62] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (63) Project [codegen id : 9] -Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (64) ReusedExchange [Reuses operator id: 31] -Output [5]: [c_customer_sk#69, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] (65) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#69] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (66) Project [codegen id : 9] -Output [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_customer_sk#69, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73] +Output [13]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] (67) ReusedExchange [Reuses operator id: 37] -Output [4]: [ca_address_sk#74, ca_state#75, ca_zip#76, ca_country#77] +Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] (68) BroadcastHashJoin [codegen id : 9] -Left keys [3]: [c_current_addr_sk#70, c_birth_country#73, s_zip#61] -Right keys [3]: [ca_address_sk#74, upper(ca_country#77), ca_zip#76] +Left keys [3]: [c_current_addr_sk#26, c_birth_country#29, s_zip#16] +Right keys [3]: [ca_address_sk#31, upper(ca_country#34), ca_zip#33] Join condition: None (69) Project [codegen id : 9] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, 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#75] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67, c_current_addr_sk#70, c_first_name#71, c_last_name#72, c_birth_country#73, ca_address_sk#74, ca_state#75, ca_zip#76, ca_country#77] +Output [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#27, c_last_name#28, ca_state#32] +Input [17]: [ss_net_paid#5, s_store_name#13, s_state#15, s_zip#16, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29, ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] (70) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, 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#75] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, 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#55))] -Aggregate Attributes [1]: [sum#78] -Results [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] +Input [11]: [ss_net_paid#5, s_store_name#13, s_state#15, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23, c_first_name#27, c_last_name#28, ca_state#32] +Keys [10]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#52] +Results [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#53] (71) Exchange -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] -Arguments: hashpartitioning(c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#53] +Arguments: hashpartitioning(c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, 5), ENSURE_REQUIREMENTS, [id=#54] (72) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64, sum#79] -Keys [10]: [c_last_name#72, c_first_name#71, s_store_name#59, ca_state#75, s_state#60, i_color#65, i_current_price#63, i_manager_id#67, i_units#66, i_size#64] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#81] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#81,17,2) AS netpaid#40] +Input [11]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20, sum#53] +Keys [10]: [c_last_name#28, c_first_name#27, s_store_name#13, ca_state#32, s_state#15, i_color#21, i_current_price#19, i_manager_id#23, i_units#22, i_size#20] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] (73) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#82, count#83] -Results [2]: [sum#84, count#85] +Aggregate Attributes [2]: [sum#55, count#56] +Results [2]: [sum#57, count#58] (74) Exchange -Input [2]: [sum#84, count#85] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#86] +Input [2]: [sum#57, count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] (75) HashAggregate [codegen id : 11] -Input [2]: [sum#84, count#85] +Input [2]: [sum#57, count#58] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#87] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#87)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#88] +Aggregate Attributes [1]: [avg(netpaid#40)#60] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#60)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#61] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt index bad2f1cbc6589..b55e5641a679a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt @@ -205,200 +205,200 @@ Aggregate Attributes [4]: [avg(agg1#22)#43, avg(UnscaledValue(agg2#23))#44, avg( Results [7]: [i_item_id#20, s_state#17, 0 AS g_state#47, avg(agg1#22)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#23))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#24))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#25))#46 / 100.0) as decimal(11,6)) AS agg4#51] (29) Scan parquet default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (31) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (32) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#60] +Output [1]: [cd_demo_sk#10] (33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#60] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (34) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (35) Scan parquet default.store -Output [2]: [s_store_sk#61, s_state#62] +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#61, s_state#62] +Input [2]: [s_store_sk#16, s_state#17] (37) Filter [codegen id : 8] -Input [2]: [s_store_sk#61, s_state#62] -Condition : ((isnotnull(s_state#62) AND (s_state#62 = TN)) AND isnotnull(s_store_sk#61)) +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) (38) Project [codegen id : 8] -Output [1]: [s_store_sk#61] -Input [2]: [s_store_sk#61, s_state#62] +Output [1]: [s_store_sk#16] +Input [2]: [s_store_sk#16, s_state#17] (39) BroadcastExchange -Input [1]: [s_store_sk#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [s_store_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#61] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, s_store_sk#61] +Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, s_store_sk#16] (42) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#64] +Output [1]: [d_date_sk#15] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (44) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#64] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#15] (45) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#65, i_item_id#66] +Output [2]: [i_item_sk#19, i_item_id#20] (46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#65] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (47) Project [codegen id : 11] -Output [5]: [i_item_id#66, ss_quantity#55 AS agg1#22, ss_list_price#56 AS agg2#23, ss_coupon_amt#58 AS agg3#24, ss_sales_price#57 AS agg4#25] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#66] +Output [5]: [i_item_id#20, ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19, i_item_id#20] (48) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#66, agg1#22, agg2#23, agg3#24, agg4#25] -Keys [1]: [i_item_id#66] +Input [5]: [i_item_id#20, agg1#22, agg2#23, agg3#24, agg4#25] +Keys [1]: [i_item_id#20] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74] -Results [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] +Aggregate Attributes [8]: [sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Results [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] (49) Exchange -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Arguments: hashpartitioning(i_item_id#66, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#69] (50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Keys [1]: [i_item_id#66] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Keys [1]: [i_item_id#20] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#84, avg(UnscaledValue(agg2#23))#85, avg(UnscaledValue(agg3#24))#86, avg(UnscaledValue(agg4#25))#87] -Results [7]: [i_item_id#66, null AS s_state#88, 1 AS g_state#89, avg(agg1#22)#84 AS agg1#90, cast((avg(UnscaledValue(agg2#23))#85 / 100.0) as decimal(11,6)) AS agg2#91, cast((avg(UnscaledValue(agg3#24))#86 / 100.0) as decimal(11,6)) AS agg3#92, cast((avg(UnscaledValue(agg4#25))#87 / 100.0) as decimal(11,6)) AS agg4#93] +Aggregate Attributes [4]: [avg(agg1#22)#70, avg(UnscaledValue(agg2#23))#71, avg(UnscaledValue(agg3#24))#72, avg(UnscaledValue(agg4#25))#73] +Results [7]: [i_item_id#20, null AS s_state#74, 1 AS g_state#75, avg(agg1#22)#70 AS agg1#76, cast((avg(UnscaledValue(agg2#23))#71 / 100.0) as decimal(11,6)) AS agg2#77, cast((avg(UnscaledValue(agg3#24))#72 / 100.0) as decimal(11,6)) AS agg3#78, cast((avg(UnscaledValue(agg4#25))#73 / 100.0) as decimal(11,6)) AS agg4#79] (51) Scan parquet default.store_sales -Output [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (53) Filter [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Condition : ((isnotnull(ss_cdemo_sk#95) AND isnotnull(ss_store_sk#96)) AND isnotnull(ss_item_sk#94)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (54) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#102] +Output [1]: [cd_demo_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#102] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Input [9]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, cd_demo_sk#102] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (57) ReusedExchange [Reuses operator id: 39] -Output [1]: [s_store_sk#103] +Output [1]: [s_store_sk#16] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#96] -Right keys [1]: [s_store_sk#103] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (59) Project [codegen id : 17] -Output [6]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Input [8]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, s_store_sk#103] +Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, s_store_sk#16] (60) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#104] +Output [1]: [d_date_sk#15] (61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#101] -Right keys [1]: [d_date_sk#104] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (62) Project [codegen id : 17] -Output [5]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [7]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, d_date_sk#104] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#15] (63) Scan parquet default.item -Output [1]: [i_item_sk#105] +Output [1]: [i_item_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#105] +Input [1]: [i_item_sk#19] (65) Filter [codegen id : 16] -Input [1]: [i_item_sk#105] -Condition : isnotnull(i_item_sk#105) +Input [1]: [i_item_sk#19] +Condition : isnotnull(i_item_sk#19) (66) BroadcastExchange -Input [1]: [i_item_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#106] +Input [1]: [i_item_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#94] -Right keys [1]: [i_item_sk#105] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (68) Project [codegen id : 17] -Output [4]: [ss_quantity#97 AS agg1#22, ss_list_price#98 AS agg2#23, ss_coupon_amt#100 AS agg3#24, ss_sales_price#99 AS agg4#25] -Input [6]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, i_item_sk#105] +Output [4]: [ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19] (69) HashAggregate [codegen id : 17] Input [4]: [agg1#22, agg2#23, agg3#24, agg4#25] Keys: [] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Results [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Aggregate Attributes [8]: [sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88] +Results [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] (70) Exchange -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (71) HashAggregate [codegen id : 18] -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] Keys: [] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#124, avg(UnscaledValue(agg2#23))#125, avg(UnscaledValue(agg3#24))#126, avg(UnscaledValue(agg4#25))#127] -Results [7]: [null AS i_item_id#128, null AS s_state#129, 1 AS g_state#130, avg(agg1#22)#124 AS agg1#131, cast((avg(UnscaledValue(agg2#23))#125 / 100.0) as decimal(11,6)) AS agg2#132, cast((avg(UnscaledValue(agg3#24))#126 / 100.0) as decimal(11,6)) AS agg3#133, cast((avg(UnscaledValue(agg4#25))#127 / 100.0) as decimal(11,6)) AS agg4#134] +Aggregate Attributes [4]: [avg(agg1#22)#98, avg(UnscaledValue(agg2#23))#99, avg(UnscaledValue(agg3#24))#100, avg(UnscaledValue(agg4#25))#101] +Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(agg1#22)#98 AS agg1#105, cast((avg(UnscaledValue(agg2#23))#99 / 100.0) as decimal(11,6)) AS agg2#106, cast((avg(UnscaledValue(agg3#24))#100 / 100.0) as decimal(11,6)) AS agg3#107, cast((avg(UnscaledValue(agg4#25))#101 / 100.0) as decimal(11,6)) AS agg4#108] (72) Union @@ -417,29 +417,29 @@ BroadcastExchange (78) (74) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#135] +Output [2]: [d_date_sk#15, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_year#135] +Input [2]: [d_date_sk#15, d_year#109] (76) Filter [codegen id : 1] -Input [2]: [d_date_sk#15, d_year#135] -Condition : ((isnotnull(d_year#135) AND (d_year#135 = 1998)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#15, d_year#109] +Condition : ((isnotnull(d_year#109) AND (d_year#109 = 1998)) AND isnotnull(d_date_sk#15)) (77) Project [codegen id : 1] Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#135] +Input [2]: [d_date_sk#15, d_year#109] (78) BroadcastExchange Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#136] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index 5fc80e2e86930..60b1498c4e6d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -205,200 +205,200 @@ Aggregate Attributes [4]: [avg(agg1#22)#43, avg(UnscaledValue(agg2#23))#44, avg( Results [7]: [i_item_id#20, s_state#17, 0 AS g_state#47, avg(agg1#22)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#23))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#24))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#25))#46 / 100.0) as decimal(11,6)) AS agg4#51] (29) Scan parquet default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (31) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (32) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#60] +Output [1]: [cd_demo_sk#10] (33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#60] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (34) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (35) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#15] (36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (37) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#15] (38) Scan parquet default.store -Output [2]: [s_store_sk#62, s_state#63] +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#62, s_state#63] +Input [2]: [s_store_sk#16, s_state#17] (40) Filter [codegen id : 9] -Input [2]: [s_store_sk#62, s_state#63] -Condition : ((isnotnull(s_state#63) AND (s_state#63 = TN)) AND isnotnull(s_store_sk#62)) +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) (41) Project [codegen id : 9] -Output [1]: [s_store_sk#62] -Input [2]: [s_store_sk#62, s_state#63] +Output [1]: [s_store_sk#16] +Input [2]: [s_store_sk#16, s_state#17] (42) BroadcastExchange -Input [1]: [s_store_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#64] +Input [1]: [s_store_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#62] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (44) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] (45) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#65, i_item_id#66] +Output [2]: [i_item_sk#19, i_item_id#20] (46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#65] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (47) Project [codegen id : 11] -Output [5]: [i_item_id#66, ss_quantity#55 AS agg1#22, ss_list_price#56 AS agg2#23, ss_coupon_amt#58 AS agg3#24, ss_sales_price#57 AS agg4#25] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#66] +Output [5]: [i_item_id#20, ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19, i_item_id#20] (48) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#66, agg1#22, agg2#23, agg3#24, agg4#25] -Keys [1]: [i_item_id#66] +Input [5]: [i_item_id#20, agg1#22, agg2#23, agg3#24, agg4#25] +Keys [1]: [i_item_id#20] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74] -Results [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] +Aggregate Attributes [8]: [sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Results [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] (49) Exchange -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Arguments: hashpartitioning(i_item_id#66, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#69] (50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#66, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Keys [1]: [i_item_id#66] +Input [9]: [i_item_id#20, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Keys [1]: [i_item_id#20] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#84, avg(UnscaledValue(agg2#23))#85, avg(UnscaledValue(agg3#24))#86, avg(UnscaledValue(agg4#25))#87] -Results [7]: [i_item_id#66, null AS s_state#88, 1 AS g_state#89, avg(agg1#22)#84 AS agg1#90, cast((avg(UnscaledValue(agg2#23))#85 / 100.0) as decimal(11,6)) AS agg2#91, cast((avg(UnscaledValue(agg3#24))#86 / 100.0) as decimal(11,6)) AS agg3#92, cast((avg(UnscaledValue(agg4#25))#87 / 100.0) as decimal(11,6)) AS agg4#93] +Aggregate Attributes [4]: [avg(agg1#22)#70, avg(UnscaledValue(agg2#23))#71, avg(UnscaledValue(agg3#24))#72, avg(UnscaledValue(agg4#25))#73] +Results [7]: [i_item_id#20, null AS s_state#74, 1 AS g_state#75, avg(agg1#22)#70 AS agg1#76, cast((avg(UnscaledValue(agg2#23))#71 / 100.0) as decimal(11,6)) AS agg2#77, cast((avg(UnscaledValue(agg3#24))#72 / 100.0) as decimal(11,6)) AS agg3#78, cast((avg(UnscaledValue(agg4#25))#73 / 100.0) as decimal(11,6)) AS agg4#79] (51) Scan parquet default.store_sales -Output [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] (53) Filter [codegen id : 17] -Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Condition : ((isnotnull(ss_cdemo_sk#95) AND isnotnull(ss_store_sk#96)) AND isnotnull(ss_item_sk#94)) +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) (54) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#102] +Output [1]: [cd_demo_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#102] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -Input [9]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, cd_demo_sk#102] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (57) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#15] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#101] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (59) Project [codegen id : 17] -Output [6]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [8]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, d_date_sk#103] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#15] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [s_store_sk#104] +Output [1]: [s_store_sk#16] (61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#96] -Right keys [1]: [s_store_sk#104] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#16] Join condition: None (62) Project [codegen id : 17] -Output [5]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -Input [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, s_store_sk#104] +Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] (63) Scan parquet default.item -Output [1]: [i_item_sk#105] +Output [1]: [i_item_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#105] +Input [1]: [i_item_sk#19] (65) Filter [codegen id : 16] -Input [1]: [i_item_sk#105] -Condition : isnotnull(i_item_sk#105) +Input [1]: [i_item_sk#19] +Condition : isnotnull(i_item_sk#19) (66) BroadcastExchange -Input [1]: [i_item_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#106] +Input [1]: [i_item_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#94] -Right keys [1]: [i_item_sk#105] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#19] Join condition: None (68) Project [codegen id : 17] -Output [4]: [ss_quantity#97 AS agg1#22, ss_list_price#98 AS agg2#23, ss_coupon_amt#100 AS agg3#24, ss_sales_price#99 AS agg4#25] -Input [6]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, i_item_sk#105] +Output [4]: [ss_quantity#4 AS agg1#22, ss_list_price#5 AS agg2#23, ss_coupon_amt#7 AS agg3#24, ss_sales_price#6 AS agg4#25] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#19] (69) HashAggregate [codegen id : 17] Input [4]: [agg1#22, agg2#23, agg3#24, agg4#25] Keys: [] Functions [4]: [partial_avg(agg1#22), partial_avg(UnscaledValue(agg2#23)), partial_avg(UnscaledValue(agg3#24)), partial_avg(UnscaledValue(agg4#25))] -Aggregate Attributes [8]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Results [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Aggregate Attributes [8]: [sum#81, count#82, sum#83, count#84, sum#85, count#86, sum#87, count#88] +Results [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] (70) Exchange -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (71) HashAggregate [codegen id : 18] -Input [8]: [sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122] +Input [8]: [sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96] Keys: [] Functions [4]: [avg(agg1#22), avg(UnscaledValue(agg2#23)), avg(UnscaledValue(agg3#24)), avg(UnscaledValue(agg4#25))] -Aggregate Attributes [4]: [avg(agg1#22)#124, avg(UnscaledValue(agg2#23))#125, avg(UnscaledValue(agg3#24))#126, avg(UnscaledValue(agg4#25))#127] -Results [7]: [null AS i_item_id#128, null AS s_state#129, 1 AS g_state#130, avg(agg1#22)#124 AS agg1#131, cast((avg(UnscaledValue(agg2#23))#125 / 100.0) as decimal(11,6)) AS agg2#132, cast((avg(UnscaledValue(agg3#24))#126 / 100.0) as decimal(11,6)) AS agg3#133, cast((avg(UnscaledValue(agg4#25))#127 / 100.0) as decimal(11,6)) AS agg4#134] +Aggregate Attributes [4]: [avg(agg1#22)#98, avg(UnscaledValue(agg2#23))#99, avg(UnscaledValue(agg3#24))#100, avg(UnscaledValue(agg4#25))#101] +Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(agg1#22)#98 AS agg1#105, cast((avg(UnscaledValue(agg2#23))#99 / 100.0) as decimal(11,6)) AS agg2#106, cast((avg(UnscaledValue(agg3#24))#100 / 100.0) as decimal(11,6)) AS agg3#107, cast((avg(UnscaledValue(agg4#25))#101 / 100.0) as decimal(11,6)) AS agg4#108] (72) Union @@ -417,29 +417,29 @@ BroadcastExchange (78) (74) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#135] +Output [2]: [d_date_sk#15, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_year#135] +Input [2]: [d_date_sk#15, d_year#109] (76) Filter [codegen id : 1] -Input [2]: [d_date_sk#15, d_year#135] -Condition : ((isnotnull(d_year#135) AND (d_year#135 = 1998)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#15, d_year#109] +Condition : ((isnotnull(d_year#109) AND (d_year#109 = 1998)) AND isnotnull(d_date_sk#15)) (77) Project [codegen id : 1] Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#135] +Input [2]: [d_date_sk#15, d_year#109] (78) BroadcastExchange Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#136] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt index 528abfede9e0f..0e20331e83484 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt @@ -146,60 +146,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#22, i_category#13, i_class#12, 0 AS t_category#23, 0 AS t_class#24, 0 AS lochierarchy#25] (23) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#26, i_class#27, sum#28, sum#29] +Output [4]: [i_category#13, i_class#12, sum#26, sum#27] (24) HashAggregate [codegen id : 10] -Input [4]: [i_category#26, i_class#27, sum#28, sum#29] -Keys [2]: [i_category#26, i_class#27] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#30))#32, sum(UnscaledValue(ss_ext_sales_price#31))#33] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#30))#32,17,2) AS ss_net_profit#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#33,17,2) AS ss_ext_sales_price#35, i_category#26] +Input [4]: [i_category#13, i_class#12, sum#26, sum#27] +Keys [2]: [i_category#13, i_class#12] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#28, sum(UnscaledValue(ss_ext_sales_price#3))#29] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#28,17,2) AS ss_net_profit#30, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#29,17,2) AS ss_ext_sales_price#31, i_category#13] (25) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#34, ss_ext_sales_price#35, i_category#26] -Keys [1]: [i_category#26] -Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -Aggregate Attributes [4]: [sum#36, isEmpty#37, sum#38, isEmpty#39] -Results [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] +Input [3]: [ss_net_profit#30, ss_ext_sales_price#31, i_category#13] +Keys [1]: [i_category#13] +Functions [2]: [partial_sum(ss_net_profit#30), partial_sum(ss_ext_sales_price#31)] +Aggregate Attributes [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] +Results [5]: [i_category#13, sum#36, isEmpty#37, sum#38, isEmpty#39] (26) Exchange -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_category#26, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [i_category#13, sum#36, isEmpty#37, sum#38, isEmpty#39] +Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, [id=#40] (27) HashAggregate [codegen id : 11] -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Keys [1]: [i_category#26] -Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -Aggregate Attributes [2]: [sum(ss_net_profit#34)#45, sum(ss_ext_sales_price#35)#46] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#34)#45) / promote_precision(sum(ss_ext_sales_price#35)#46)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#47, i_category#26, null AS i_class#48, 0 AS t_category#49, 1 AS t_class#50, 1 AS lochierarchy#51] +Input [5]: [i_category#13, sum#36, isEmpty#37, sum#38, isEmpty#39] +Keys [1]: [i_category#13] +Functions [2]: [sum(ss_net_profit#30), sum(ss_ext_sales_price#31)] +Aggregate Attributes [2]: [sum(ss_net_profit#30)#41, sum(ss_ext_sales_price#31)#42] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#30)#41) / promote_precision(sum(ss_ext_sales_price#31)#42)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#43, i_category#13, null AS i_class#44, 0 AS t_category#45, 1 AS t_class#46, 1 AS lochierarchy#47] (28) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#52, i_class#53, sum#54, sum#55] +Output [4]: [i_category#13, i_class#12, sum#48, sum#49] (29) HashAggregate [codegen id : 16] -Input [4]: [i_category#52, i_class#53, sum#54, sum#55] -Keys [2]: [i_category#52, i_class#53] -Functions [2]: [sum(UnscaledValue(ss_net_profit#56)), sum(UnscaledValue(ss_ext_sales_price#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#56))#58, sum(UnscaledValue(ss_ext_sales_price#57))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#58,17,2) AS ss_net_profit#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#57))#59,17,2) AS ss_ext_sales_price#35] +Input [4]: [i_category#13, i_class#12, sum#48, sum#49] +Keys [2]: [i_category#13, i_class#12] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#28, sum(UnscaledValue(ss_ext_sales_price#3))#29] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#28,17,2) AS ss_net_profit#30, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#29,17,2) AS ss_ext_sales_price#31] (30) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#34, ss_ext_sales_price#35] +Input [2]: [ss_net_profit#30, ss_ext_sales_price#31] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -Aggregate Attributes [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Results [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [2]: [partial_sum(ss_net_profit#30), partial_sum(ss_ext_sales_price#31)] +Aggregate Attributes [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +Results [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] (31) Exchange -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] (32) HashAggregate [codegen id : 17] -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] Keys: [] -Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -Aggregate Attributes [2]: [sum(ss_net_profit#34)#69, sum(ss_ext_sales_price#35)#70] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#34)#69) / promote_precision(sum(ss_ext_sales_price#35)#70)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#71, null AS i_category#72, null AS i_class#73, 1 AS t_category#74, 1 AS t_class#75, 2 AS lochierarchy#76] +Functions [2]: [sum(ss_net_profit#30), sum(ss_ext_sales_price#31)] +Aggregate Attributes [2]: [sum(ss_net_profit#30)#59, sum(ss_ext_sales_price#31)#60] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#30)#59) / promote_precision(sum(ss_ext_sales_price#31)#60)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#61, null AS i_category#62, null AS i_class#63, 1 AS t_category#64, 1 AS t_class#65, 2 AS lochierarchy#66] (33) Union @@ -212,34 +212,34 @@ Results [6]: [gross_margin#22, i_category#13, i_class#12, t_category#23, t_class (35) Exchange Input [6]: [gross_margin#22, i_category#13, i_class#12, t_category#23, t_class#24, lochierarchy#25] -Arguments: hashpartitioning(gross_margin#22, i_category#13, i_class#12, t_category#23, t_class#24, lochierarchy#25, 5), ENSURE_REQUIREMENTS, [id=#77] +Arguments: hashpartitioning(gross_margin#22, i_category#13, i_class#12, t_category#23, t_class#24, lochierarchy#25, 5), ENSURE_REQUIREMENTS, [id=#67] (36) HashAggregate [codegen id : 19] Input [6]: [gross_margin#22, i_category#13, i_class#12, t_category#23, t_class#24, lochierarchy#25] Keys [6]: [gross_margin#22, i_category#13, i_class#12, t_category#23, t_class#24, lochierarchy#25] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, CASE WHEN (t_class#24 = 0) THEN i_category#13 END AS _w0#78] +Results [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, CASE WHEN (t_class#24 = 0) THEN i_category#13 END AS _w0#68] (37) Exchange -Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#78] -Arguments: hashpartitioning(lochierarchy#25, _w0#78, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#68] +Arguments: hashpartitioning(lochierarchy#25, _w0#68, 5), ENSURE_REQUIREMENTS, [id=#69] (38) Sort [codegen id : 20] -Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#78] -Arguments: [lochierarchy#25 ASC NULLS FIRST, _w0#78 ASC NULLS FIRST, gross_margin#22 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#68] +Arguments: [lochierarchy#25 ASC NULLS FIRST, _w0#68 ASC NULLS FIRST, gross_margin#22 ASC NULLS FIRST], false, 0 (39) Window -Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#78] -Arguments: [rank(gross_margin#22) windowspecdefinition(lochierarchy#25, _w0#78, gross_margin#22 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#80], [lochierarchy#25, _w0#78], [gross_margin#22 ASC NULLS FIRST] +Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#68] +Arguments: [rank(gross_margin#22) windowspecdefinition(lochierarchy#25, _w0#68, gross_margin#22 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#70], [lochierarchy#25, _w0#68], [gross_margin#22 ASC NULLS FIRST] (40) Project [codegen id : 21] -Output [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#80] -Input [6]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#78, rank_within_parent#80] +Output [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70] +Input [6]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#68, rank_within_parent#70] (41) TakeOrderedAndProject -Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#80] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#80 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#80] +Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70] +Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70] ===== Subqueries ===== @@ -252,25 +252,25 @@ BroadcastExchange (46) (42) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_year#81] +Output [2]: [d_date_sk#7, d_year#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#7, d_year#81] +Input [2]: [d_date_sk#7, d_year#71] (44) Filter [codegen id : 1] -Input [2]: [d_date_sk#7, d_year#81] -Condition : ((isnotnull(d_year#81) AND (d_year#81 = 2001)) AND isnotnull(d_date_sk#7)) +Input [2]: [d_date_sk#7, d_year#71] +Condition : ((isnotnull(d_year#71) AND (d_year#71 = 2001)) AND isnotnull(d_date_sk#7)) (45) Project [codegen id : 1] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_year#81] +Input [2]: [d_date_sk#7, d_year#71] (46) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#82] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index 573545537e868..5470bf61ac502 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -146,60 +146,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#22, i_category#10, i_class#9, 0 AS t_category#23, 0 AS t_class#24, 0 AS lochierarchy#25] (23) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#26, i_class#27, sum#28, sum#29] +Output [4]: [i_category#10, i_class#9, sum#26, sum#27] (24) HashAggregate [codegen id : 10] -Input [4]: [i_category#26, i_class#27, sum#28, sum#29] -Keys [2]: [i_category#26, i_class#27] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#30))#32, sum(UnscaledValue(ss_ext_sales_price#31))#33] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#30))#32,17,2) AS ss_net_profit#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#33,17,2) AS ss_ext_sales_price#35, i_category#26] +Input [4]: [i_category#10, i_class#9, sum#26, sum#27] +Keys [2]: [i_category#10, i_class#9] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#28, sum(UnscaledValue(ss_ext_sales_price#3))#29] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#28,17,2) AS ss_net_profit#30, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#29,17,2) AS ss_ext_sales_price#31, i_category#10] (25) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#34, ss_ext_sales_price#35, i_category#26] -Keys [1]: [i_category#26] -Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -Aggregate Attributes [4]: [sum#36, isEmpty#37, sum#38, isEmpty#39] -Results [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] +Input [3]: [ss_net_profit#30, ss_ext_sales_price#31, i_category#10] +Keys [1]: [i_category#10] +Functions [2]: [partial_sum(ss_net_profit#30), partial_sum(ss_ext_sales_price#31)] +Aggregate Attributes [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] +Results [5]: [i_category#10, sum#36, isEmpty#37, sum#38, isEmpty#39] (26) Exchange -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_category#26, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [i_category#10, sum#36, isEmpty#37, sum#38, isEmpty#39] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#40] (27) HashAggregate [codegen id : 11] -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Keys [1]: [i_category#26] -Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -Aggregate Attributes [2]: [sum(ss_net_profit#34)#45, sum(ss_ext_sales_price#35)#46] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#34)#45) / promote_precision(sum(ss_ext_sales_price#35)#46)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#47, i_category#26, null AS i_class#48, 0 AS t_category#49, 1 AS t_class#50, 1 AS lochierarchy#51] +Input [5]: [i_category#10, sum#36, isEmpty#37, sum#38, isEmpty#39] +Keys [1]: [i_category#10] +Functions [2]: [sum(ss_net_profit#30), sum(ss_ext_sales_price#31)] +Aggregate Attributes [2]: [sum(ss_net_profit#30)#41, sum(ss_ext_sales_price#31)#42] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#30)#41) / promote_precision(sum(ss_ext_sales_price#31)#42)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#43, i_category#10, null AS i_class#44, 0 AS t_category#45, 1 AS t_class#46, 1 AS lochierarchy#47] (28) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#52, i_class#53, sum#54, sum#55] +Output [4]: [i_category#10, i_class#9, sum#48, sum#49] (29) HashAggregate [codegen id : 16] -Input [4]: [i_category#52, i_class#53, sum#54, sum#55] -Keys [2]: [i_category#52, i_class#53] -Functions [2]: [sum(UnscaledValue(ss_net_profit#56)), sum(UnscaledValue(ss_ext_sales_price#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#56))#58, sum(UnscaledValue(ss_ext_sales_price#57))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#58,17,2) AS ss_net_profit#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#57))#59,17,2) AS ss_ext_sales_price#35] +Input [4]: [i_category#10, i_class#9, sum#48, sum#49] +Keys [2]: [i_category#10, i_class#9] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#28, sum(UnscaledValue(ss_ext_sales_price#3))#29] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#28,17,2) AS ss_net_profit#30, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#29,17,2) AS ss_ext_sales_price#31] (30) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#34, ss_ext_sales_price#35] +Input [2]: [ss_net_profit#30, ss_ext_sales_price#31] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -Aggregate Attributes [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Results [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [2]: [partial_sum(ss_net_profit#30), partial_sum(ss_ext_sales_price#31)] +Aggregate Attributes [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +Results [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] (31) Exchange -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] (32) HashAggregate [codegen id : 17] -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] Keys: [] -Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -Aggregate Attributes [2]: [sum(ss_net_profit#34)#69, sum(ss_ext_sales_price#35)#70] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#34)#69) / promote_precision(sum(ss_ext_sales_price#35)#70)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#71, null AS i_category#72, null AS i_class#73, 1 AS t_category#74, 1 AS t_class#75, 2 AS lochierarchy#76] +Functions [2]: [sum(ss_net_profit#30), sum(ss_ext_sales_price#31)] +Aggregate Attributes [2]: [sum(ss_net_profit#30)#59, sum(ss_ext_sales_price#31)#60] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#30)#59) / promote_precision(sum(ss_ext_sales_price#31)#60)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#61, null AS i_category#62, null AS i_class#63, 1 AS t_category#64, 1 AS t_class#65, 2 AS lochierarchy#66] (33) Union @@ -212,34 +212,34 @@ Results [6]: [gross_margin#22, i_category#10, i_class#9, t_category#23, t_class# (35) Exchange Input [6]: [gross_margin#22, i_category#10, i_class#9, t_category#23, t_class#24, lochierarchy#25] -Arguments: hashpartitioning(gross_margin#22, i_category#10, i_class#9, t_category#23, t_class#24, lochierarchy#25, 5), ENSURE_REQUIREMENTS, [id=#77] +Arguments: hashpartitioning(gross_margin#22, i_category#10, i_class#9, t_category#23, t_class#24, lochierarchy#25, 5), ENSURE_REQUIREMENTS, [id=#67] (36) HashAggregate [codegen id : 19] Input [6]: [gross_margin#22, i_category#10, i_class#9, t_category#23, t_class#24, lochierarchy#25] Keys [6]: [gross_margin#22, i_category#10, i_class#9, t_category#23, t_class#24, lochierarchy#25] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, CASE WHEN (t_class#24 = 0) THEN i_category#10 END AS _w0#78] +Results [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, CASE WHEN (t_class#24 = 0) THEN i_category#10 END AS _w0#68] (37) Exchange -Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#78] -Arguments: hashpartitioning(lochierarchy#25, _w0#78, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#68] +Arguments: hashpartitioning(lochierarchy#25, _w0#68, 5), ENSURE_REQUIREMENTS, [id=#69] (38) Sort [codegen id : 20] -Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#78] -Arguments: [lochierarchy#25 ASC NULLS FIRST, _w0#78 ASC NULLS FIRST, gross_margin#22 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#68] +Arguments: [lochierarchy#25 ASC NULLS FIRST, _w0#68 ASC NULLS FIRST, gross_margin#22 ASC NULLS FIRST], false, 0 (39) Window -Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#78] -Arguments: [rank(gross_margin#22) windowspecdefinition(lochierarchy#25, _w0#78, gross_margin#22 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#80], [lochierarchy#25, _w0#78], [gross_margin#22 ASC NULLS FIRST] +Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#68] +Arguments: [rank(gross_margin#22) windowspecdefinition(lochierarchy#25, _w0#68, gross_margin#22 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#70], [lochierarchy#25, _w0#68], [gross_margin#22 ASC NULLS FIRST] (40) Project [codegen id : 21] -Output [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#80] -Input [6]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#78, rank_within_parent#80] +Output [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70] +Input [6]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#68, rank_within_parent#70] (41) TakeOrderedAndProject -Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#80] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#80 ASC NULLS FIRST], [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#80] +Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70] +Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70] ===== Subqueries ===== @@ -252,25 +252,25 @@ BroadcastExchange (46) (42) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_year#81] +Output [2]: [d_date_sk#7, d_year#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#7, d_year#81] +Input [2]: [d_date_sk#7, d_year#71] (44) Filter [codegen id : 1] -Input [2]: [d_date_sk#7, d_year#81] -Condition : ((isnotnull(d_year#81) AND (d_year#81 = 2001)) AND isnotnull(d_date_sk#7)) +Input [2]: [d_date_sk#7, d_year#71] +Condition : ((isnotnull(d_year#71) AND (d_year#71 = 2001)) AND isnotnull(d_date_sk#7)) (45) Project [codegen id : 1] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_year#81] +Input [2]: [d_date_sk#7, d_year#71] (46) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#82] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 1e64471e096cb..4566f30b27d04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -207,77 +207,77 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] Keys [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33] Functions [1]: [sum(UnscaledValue(ss_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#36] -Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#36,17,2) AS sum_sales#37] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#35))#21] +Results [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(ss_sales_price#35))#21,17,2) AS sum_sales#22] (36) Exchange -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#36] (37) Sort [codegen id : 21] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (38) Window -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#28, i_brand#29, s_store_name#30, s_company_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#22 AS sum_sales#38, rn#37] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#22, rn#37] (40) Exchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 23] -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] +Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_name#30 ASC NULLS FIRST, s_company_name#31 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#37 + 1)] Join condition: None (43) Project [codegen id : 24] -Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37] -Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38] +Input [15]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#38, rn#37] (44) ReusedExchange [Reuses operator id: 36] -Output [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] +Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] (45) Sort [codegen id : 33] -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (46) Window -Input [7]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47] -Arguments: [rank(d_year#45, d_moy#46) windowspecdefinition(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#48], [i_category#41, i_brand#42, s_store_name#43, s_company_name#44], [d_year#45 ASC NULLS FIRST, d_moy#46 ASC NULLS FIRST] +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, s_store_name#42, s_company_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Input [8]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, d_year#45, d_moy#46, sum_sales#47, rn#48] +Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#22 AS sum_sales#47, rn#46] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#22, rn#46] (48) Exchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: hashpartitioning(i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] +Arguments: hashpartitioning(i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] (49) Sort [codegen id : 35] -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, s_store_name#43 ASC NULLS FIRST, s_company_name#44 ASC NULLS FIRST, (rn#48 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, s_store_name#42 ASC NULLS FIRST, s_company_name#43 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [5]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, rn#25] -Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] +Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#46 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Output [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, sum_sales#38 AS psum#49, sum_sales#47 AS nsum#50] +Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#26, rn#25, sum_sales#38, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#47, rn#46] (52) TakeOrderedAndProject -Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#50, nsum#51] +Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index fcb93277bc76a..21944f91237a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -180,69 +180,69 @@ Output [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_ye Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum#32] Keys [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31] Functions [1]: [sum(UnscaledValue(ss_sales_price#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#33))#34] -Results [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(ss_sales_price#33))#34,17,2) AS sum_sales#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#33))#20] +Results [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(ss_sales_price#33))#20,17,2) AS sum_sales#21] (31) Exchange -Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: hashpartitioning(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: hashpartitioning(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, 5), ENSURE_REQUIREMENTS, [id=#34] (32) Sort [codegen id : 13] -Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35] +Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21] Arguments: [i_category#26 ASC NULLS FIRST, i_brand#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST, s_company_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 (33) Window -Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +Input [7]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#26, i_brand#27, s_store_name#28, s_company_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] -Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Output [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#21 AS sum_sales#36, rn#35] +Input [8]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35] (35) BroadcastExchange -Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#38] +Input [6]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#36, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [id=#37] (36) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#37 + 1)] +Right keys [5]: [i_category#26, i_brand#27, s_store_name#28, s_company_name#29, (rn#35 + 1)] Join condition: None (37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#35, rn#37] +Output [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36] +Input [15]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#26, i_brand#27, s_store_name#28, s_company_name#29, sum_sales#36, rn#35] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21] (39) Sort [codegen id : 20] -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, s_store_name#41 ASC NULLS FIRST, s_company_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, s_store_name#40 ASC NULLS FIRST, s_company_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#39, i_brand#40, s_store_name#41, s_company_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [7]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21] +Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#44], [i_category#38, i_brand#39, s_store_name#40, s_company_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Input [8]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#21 AS sum_sales#45, rn#44] +Input [8]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, d_year#42, d_moy#43, sum_sales#21, rn#44] (42) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#47] +Input [6]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [id=#46] (43) BroadcastHashJoin [codegen id : 22] Left keys [5]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, rn#24] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#41, s_company_name#42, (rn#46 - 1)] +Right keys [5]: [i_category#38, i_brand#39, s_store_name#40, s_company_name#41, (rn#44 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#48, sum_sales#45 AS nsum#49] -Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, s_store_name#41, s_company_name#42, sum_sales#45, rn#46] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#47, sum_sales#45 AS nsum#48] +Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, s_store_name#40, s_company_name#41, sum_sales#45, rn#44] (45) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#48, nsum#49] +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index c588559846584..740ea0f9ebbd2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -141,50 +141,50 @@ Input [4]: [item_sk#11, d_date#6, sumws#12, rk#14] Arguments: [item_sk#11 ASC NULLS FIRST], false, 0 (16) ReusedExchange [Reuses operator id: 10] -Output [4]: [item_sk#16, d_date#17, sumws#18, ws_item_sk#19] +Output [4]: [item_sk#11, d_date#16, sumws#12, ws_item_sk#17] (17) Sort [codegen id : 10] -Input [4]: [item_sk#16, d_date#17, sumws#18, ws_item_sk#19] -Arguments: [ws_item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#11, d_date#16, sumws#12, ws_item_sk#17] +Arguments: [ws_item_sk#17 ASC NULLS FIRST, d_date#16 ASC NULLS FIRST], false, 0 (18) Window -Input [4]: [item_sk#16, d_date#17, sumws#18, ws_item_sk#19] -Arguments: [row_number() windowspecdefinition(ws_item_sk#19, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#20], [ws_item_sk#19], [d_date#17 ASC NULLS FIRST] +Input [4]: [item_sk#11, d_date#16, sumws#12, ws_item_sk#17] +Arguments: [row_number() windowspecdefinition(ws_item_sk#17, d_date#16 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#18], [ws_item_sk#17], [d_date#16 ASC NULLS FIRST] (19) Project [codegen id : 11] -Output [3]: [item_sk#16, sumws#18, rk#20] -Input [5]: [item_sk#16, d_date#17, sumws#18, ws_item_sk#19, rk#20] +Output [3]: [item_sk#11 AS item_sk#19, sumws#12 AS sumws#20, rk#18] +Input [5]: [item_sk#11, d_date#16, sumws#12, ws_item_sk#17, rk#18] (20) Exchange -Input [3]: [item_sk#16, sumws#18, rk#20] -Arguments: hashpartitioning(item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [item_sk#19, sumws#20, rk#18] +Arguments: hashpartitioning(item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (21) Sort [codegen id : 12] -Input [3]: [item_sk#16, sumws#18, rk#20] -Arguments: [item_sk#16 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#19, sumws#20, rk#18] +Arguments: [item_sk#19 ASC NULLS FIRST], false, 0 (22) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#11] -Right keys [1]: [item_sk#16] -Join condition: (rk#14 >= rk#20) +Right keys [1]: [item_sk#19] +Join condition: (rk#14 >= rk#18) (23) Project [codegen id : 13] -Output [4]: [item_sk#11, d_date#6, sumws#12, sumws#18] -Input [7]: [item_sk#11, d_date#6, sumws#12, rk#14, item_sk#16, sumws#18, rk#20] +Output [4]: [item_sk#11, d_date#6, sumws#12, sumws#20] +Input [7]: [item_sk#11, d_date#6, sumws#12, rk#14, item_sk#19, sumws#20, rk#18] (24) HashAggregate [codegen id : 13] -Input [4]: [item_sk#11, d_date#6, sumws#12, sumws#18] +Input [4]: [item_sk#11, d_date#6, sumws#12, sumws#20] Keys [3]: [item_sk#11, d_date#6, sumws#12] -Functions [1]: [partial_sum(sumws#18)] +Functions [1]: [partial_sum(sumws#20)] Aggregate Attributes [2]: [sum#22, isEmpty#23] Results [5]: [item_sk#11, d_date#6, sumws#12, sum#24, isEmpty#25] (25) HashAggregate [codegen id : 13] Input [5]: [item_sk#11, d_date#6, sumws#12, sum#24, isEmpty#25] Keys [3]: [item_sk#11, d_date#6, sumws#12] -Functions [1]: [sum(sumws#18)] -Aggregate Attributes [1]: [sum(sumws#18)#26] -Results [3]: [item_sk#11, d_date#6, sum(sumws#18)#26 AS cume_sales#27] +Functions [1]: [sum(sumws#20)] +Aggregate Attributes [1]: [sum(sumws#20)#26] +Results [3]: [item_sk#11, d_date#6, sum(sumws#20)#26 AS cume_sales#27] (26) Exchange Input [3]: [item_sk#11, d_date#6, cume_sales#27] @@ -264,50 +264,50 @@ Input [4]: [item_sk#38, d_date#33, sumss#39, rk#41] Arguments: [item_sk#38 ASC NULLS FIRST], false, 0 (43) ReusedExchange [Reuses operator id: 37] -Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#46] +Output [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44] (44) Sort [codegen id : 24] -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#46] -Arguments: [ss_item_sk#46 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44] +Arguments: [ss_item_sk#44 ASC NULLS FIRST, d_date#43 ASC NULLS FIRST], false, 0 (45) Window -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#46] -Arguments: [row_number() windowspecdefinition(ss_item_sk#46, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [ss_item_sk#46], [d_date#44 ASC NULLS FIRST] +Input [4]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44] +Arguments: [row_number() windowspecdefinition(ss_item_sk#44, d_date#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#45], [ss_item_sk#44], [d_date#43 ASC NULLS FIRST] (46) Project [codegen id : 25] -Output [3]: [item_sk#43, sumss#45, rk#47] -Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#46, rk#47] +Output [3]: [item_sk#38 AS item_sk#46, sumss#39 AS sumss#47, rk#45] +Input [5]: [item_sk#38, d_date#43, sumss#39, ss_item_sk#44, rk#45] (47) Exchange -Input [3]: [item_sk#43, sumss#45, rk#47] -Arguments: hashpartitioning(item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [item_sk#46, sumss#47, rk#45] +Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#48] (48) Sort [codegen id : 26] -Input [3]: [item_sk#43, sumss#45, rk#47] -Arguments: [item_sk#43 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#46, sumss#47, rk#45] +Arguments: [item_sk#46 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 27] Left keys [1]: [item_sk#38] -Right keys [1]: [item_sk#43] -Join condition: (rk#41 >= rk#47) +Right keys [1]: [item_sk#46] +Join condition: (rk#41 >= rk#45) (50) Project [codegen id : 27] -Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#45] -Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#43, sumss#45, rk#47] +Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#47] +Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#46, sumss#47, rk#45] (51) HashAggregate [codegen id : 27] -Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#45] +Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#47] Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [partial_sum(sumss#45)] +Functions [1]: [partial_sum(sumss#47)] Aggregate Attributes [2]: [sum#49, isEmpty#50] Results [5]: [item_sk#38, d_date#33, sumss#39, sum#51, isEmpty#52] (52) HashAggregate [codegen id : 27] Input [5]: [item_sk#38, d_date#33, sumss#39, sum#51, isEmpty#52] Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [sum(sumss#45)] -Aggregate Attributes [1]: [sum(sumss#45)#53] -Results [3]: [item_sk#38, d_date#33, sum(sumss#45)#53 AS cume_sales#54] +Functions [1]: [sum(sumss#47)] +Aggregate Attributes [1]: [sum(sumss#47)#53] +Results [3]: [item_sk#38, d_date#33, sum(sumss#47)#53 AS cume_sales#54] (53) Exchange Input [3]: [item_sk#38, d_date#33, cume_sales#54] @@ -343,50 +343,50 @@ Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] (61) ReusedExchange [Reuses operator id: 58] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] +Output [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] (62) Sort [codegen id : 60] -Input [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] -Arguments: [item_sk#62 ASC NULLS FIRST, d_date#63 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 (63) Window -Input [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] -Arguments: [row_number() windowspecdefinition(item_sk#62, d_date#63 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#66], [item_sk#62], [d_date#63 ASC NULLS FIRST] +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#62], [item_sk#56], [d_date#57 ASC NULLS FIRST] (64) Project [codegen id : 61] -Output [4]: [item_sk#62, web_sales#64, store_sales#65, rk#66] -Input [5]: [item_sk#62, d_date#63, web_sales#64, store_sales#65, rk#66] +Output [4]: [item_sk#56 AS item_sk#63, web_sales#58 AS web_sales#64, store_sales#59 AS store_sales#65, rk#62] +Input [5]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#62] (65) SortMergeJoin [codegen id : 62] Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) +Right keys [1]: [item_sk#63] +Join condition: (rk#61 >= rk#62) (66) Project [codegen id : 62] Output [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65] -Input [9]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#61, item_sk#62, web_sales#64, store_sales#65, rk#66] +Input [9]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#61, item_sk#63, web_sales#64, store_sales#65, rk#62] (67) HashAggregate [codegen id : 62] Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65] Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] Functions [2]: [partial_max(web_sales#64), partial_max(store_sales#65)] -Aggregate Attributes [2]: [max#67, max#68] -Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#69, max#70] +Aggregate Attributes [2]: [max#66, max#67] +Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#68, max#69] (68) HashAggregate [codegen id : 62] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#69, max#70] +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#68, max#69] Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] Functions [2]: [max(web_sales#64), max(store_sales#65)] -Aggregate Attributes [2]: [max(web_sales#64)#71, max(store_sales#65)#72] -Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max(web_sales#64)#71 AS web_cumulative#73, max(store_sales#65)#72 AS store_cumulative#74] +Aggregate Attributes [2]: [max(web_sales#64)#70, max(store_sales#65)#71] +Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max(web_sales#64)#70 AS web_cumulative#72, max(store_sales#65)#71 AS store_cumulative#73] (69) Filter [codegen id : 62] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] -Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) AND (web_cumulative#73 > store_cumulative#74)) +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] +Condition : ((isnotnull(web_cumulative#72) AND isnotnull(store_cumulative#73)) AND (web_cumulative#72 > store_cumulative#73)) (70) TakeOrderedAndProject -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] -Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] +Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] ===== Subqueries ===== @@ -399,26 +399,26 @@ BroadcastExchange (75) (71) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#75] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#74] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#75] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74] (73) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#75] -Condition : (((isnotnull(d_month_seq#75) AND (d_month_seq#75 >= 1212)) AND (d_month_seq#75 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74] +Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1212)) AND (d_month_seq#74 <= 1223)) AND isnotnull(d_date_sk#5)) (74) Project [codegen id : 1] Output [2]: [d_date_sk#5, d_date#6] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#75] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#74] (75) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 7986c1f5c79f3..cf86cd670456f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -130,37 +130,37 @@ Output [4]: [item_sk#11, d_date#6, sumws#12, rk#14] Input [5]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1, rk#14] (14) ReusedExchange [Reuses operator id: 10] -Output [4]: [item_sk#15, d_date#16, sumws#17, ws_item_sk#18] +Output [4]: [item_sk#11, d_date#15, sumws#12, ws_item_sk#16] (15) Sort [codegen id : 8] -Input [4]: [item_sk#15, d_date#16, sumws#17, ws_item_sk#18] -Arguments: [ws_item_sk#18 ASC NULLS FIRST, d_date#16 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#11, d_date#15, sumws#12, ws_item_sk#16] +Arguments: [ws_item_sk#16 ASC NULLS FIRST, d_date#15 ASC NULLS FIRST], false, 0 (16) Window -Input [4]: [item_sk#15, d_date#16, sumws#17, ws_item_sk#18] -Arguments: [row_number() windowspecdefinition(ws_item_sk#18, d_date#16 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#19], [ws_item_sk#18], [d_date#16 ASC NULLS FIRST] +Input [4]: [item_sk#11, d_date#15, sumws#12, ws_item_sk#16] +Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#15 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#15 ASC NULLS FIRST] (17) Project [codegen id : 9] -Output [3]: [item_sk#15, sumws#17, rk#19] -Input [5]: [item_sk#15, d_date#16, sumws#17, ws_item_sk#18, rk#19] +Output [3]: [item_sk#11 AS item_sk#18, sumws#12 AS sumws#19, rk#17] +Input [5]: [item_sk#11, d_date#15, sumws#12, ws_item_sk#16, rk#17] (18) BroadcastExchange -Input [3]: [item_sk#15, sumws#17, rk#19] +Input [3]: [item_sk#18, sumws#19, rk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (19) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [item_sk#15] -Join condition: (rk#14 >= rk#19) +Right keys [1]: [item_sk#18] +Join condition: (rk#14 >= rk#17) (20) Project [codegen id : 10] -Output [4]: [item_sk#11, d_date#6, sumws#12, sumws#17] -Input [7]: [item_sk#11, d_date#6, sumws#12, rk#14, item_sk#15, sumws#17, rk#19] +Output [4]: [item_sk#11, d_date#6, sumws#12, sumws#19] +Input [7]: [item_sk#11, d_date#6, sumws#12, rk#14, item_sk#18, sumws#19, rk#17] (21) HashAggregate [codegen id : 10] -Input [4]: [item_sk#11, d_date#6, sumws#12, sumws#17] +Input [4]: [item_sk#11, d_date#6, sumws#12, sumws#19] Keys [3]: [item_sk#11, d_date#6, sumws#12] -Functions [1]: [partial_sum(sumws#17)] +Functions [1]: [partial_sum(sumws#19)] Aggregate Attributes [2]: [sum#21, isEmpty#22] Results [5]: [item_sk#11, d_date#6, sumws#12, sum#23, isEmpty#24] @@ -171,9 +171,9 @@ Arguments: hashpartitioning(item_sk#11, d_date#6, sumws#12, 5), ENSURE_REQUIREME (23) HashAggregate [codegen id : 11] Input [5]: [item_sk#11, d_date#6, sumws#12, sum#23, isEmpty#24] Keys [3]: [item_sk#11, d_date#6, sumws#12] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#26] -Results [3]: [item_sk#11, d_date#6, sum(sumws#17)#26 AS cume_sales#27] +Functions [1]: [sum(sumws#19)] +Aggregate Attributes [1]: [sum(sumws#19)#26] +Results [3]: [item_sk#11, d_date#6, sum(sumws#19)#26 AS cume_sales#27] (24) Exchange Input [3]: [item_sk#11, d_date#6, cume_sales#27] @@ -245,37 +245,37 @@ Output [4]: [item_sk#38, d_date#33, sumss#39, rk#41] Input [5]: [item_sk#38, d_date#33, sumss#39, ss_item_sk#29, rk#41] (39) ReusedExchange [Reuses operator id: 35] -Output [4]: [item_sk#42, d_date#43, sumss#44, ss_item_sk#45] +Output [4]: [item_sk#38, d_date#42, sumss#39, ss_item_sk#43] (40) Sort [codegen id : 20] -Input [4]: [item_sk#42, d_date#43, sumss#44, ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST, d_date#43 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#38, d_date#42, sumss#39, ss_item_sk#43] +Arguments: [ss_item_sk#43 ASC NULLS FIRST, d_date#42 ASC NULLS FIRST], false, 0 (41) Window -Input [4]: [item_sk#42, d_date#43, sumss#44, ss_item_sk#45] -Arguments: [row_number() windowspecdefinition(ss_item_sk#45, d_date#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#45], [d_date#43 ASC NULLS FIRST] +Input [4]: [item_sk#38, d_date#42, sumss#39, ss_item_sk#43] +Arguments: [row_number() windowspecdefinition(ss_item_sk#43, d_date#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [ss_item_sk#43], [d_date#42 ASC NULLS FIRST] (42) Project [codegen id : 21] -Output [3]: [item_sk#42, sumss#44, rk#46] -Input [5]: [item_sk#42, d_date#43, sumss#44, ss_item_sk#45, rk#46] +Output [3]: [item_sk#38 AS item_sk#45, sumss#39 AS sumss#46, rk#44] +Input [5]: [item_sk#38, d_date#42, sumss#39, ss_item_sk#43, rk#44] (43) BroadcastExchange -Input [3]: [item_sk#42, sumss#44, rk#46] +Input [3]: [item_sk#45, sumss#46, rk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (44) BroadcastHashJoin [codegen id : 22] Left keys [1]: [item_sk#38] -Right keys [1]: [item_sk#42] -Join condition: (rk#41 >= rk#46) +Right keys [1]: [item_sk#45] +Join condition: (rk#41 >= rk#44) (45) Project [codegen id : 22] -Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#44] -Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#42, sumss#44, rk#46] +Output [4]: [item_sk#38, d_date#33, sumss#39, sumss#46] +Input [7]: [item_sk#38, d_date#33, sumss#39, rk#41, item_sk#45, sumss#46, rk#44] (46) HashAggregate [codegen id : 22] -Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#44] +Input [4]: [item_sk#38, d_date#33, sumss#39, sumss#46] Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [partial_sum(sumss#44)] +Functions [1]: [partial_sum(sumss#46)] Aggregate Attributes [2]: [sum#48, isEmpty#49] Results [5]: [item_sk#38, d_date#33, sumss#39, sum#50, isEmpty#51] @@ -286,9 +286,9 @@ Arguments: hashpartitioning(item_sk#38, d_date#33, sumss#39, 5), ENSURE_REQUIREM (48) HashAggregate [codegen id : 23] Input [5]: [item_sk#38, d_date#33, sumss#39, sum#50, isEmpty#51] Keys [3]: [item_sk#38, d_date#33, sumss#39] -Functions [1]: [sum(sumss#44)] -Aggregate Attributes [1]: [sum(sumss#44)#53] -Results [3]: [item_sk#38, d_date#33, sum(sumss#44)#53 AS cume_sales#54] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#53] +Results [3]: [item_sk#38, d_date#33, sum(sumss#46)#53 AS cume_sales#54] (49) Exchange Input [3]: [item_sk#38, d_date#33, cume_sales#54] @@ -324,54 +324,54 @@ Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] (57) ReusedExchange [Reuses operator id: 54] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] +Output [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] (58) Sort [codegen id : 52] -Input [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] -Arguments: [item_sk#62 ASC NULLS FIRST, d_date#63 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 (59) Window -Input [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] -Arguments: [row_number() windowspecdefinition(item_sk#62, d_date#63 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#66], [item_sk#62], [d_date#63 ASC NULLS FIRST] +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#62], [item_sk#56], [d_date#57 ASC NULLS FIRST] (60) Project [codegen id : 53] -Output [4]: [item_sk#62, web_sales#64, store_sales#65, rk#66] -Input [5]: [item_sk#62, d_date#63, web_sales#64, store_sales#65, rk#66] +Output [4]: [item_sk#56 AS item_sk#63, web_sales#58 AS web_sales#64, store_sales#59 AS store_sales#65, rk#62] +Input [5]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#62] (61) BroadcastExchange -Input [4]: [item_sk#62, web_sales#64, store_sales#65, rk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#67] +Input [4]: [item_sk#63, web_sales#64, store_sales#65, rk#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (62) BroadcastHashJoin [codegen id : 54] Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) +Right keys [1]: [item_sk#63] +Join condition: (rk#61 >= rk#62) (63) Project [codegen id : 54] Output [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65] -Input [9]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#61, item_sk#62, web_sales#64, store_sales#65, rk#66] +Input [9]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, rk#61, item_sk#63, web_sales#64, store_sales#65, rk#62] (64) HashAggregate [codegen id : 54] Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_sales#64, store_sales#65] Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] Functions [2]: [partial_max(web_sales#64), partial_max(store_sales#65)] -Aggregate Attributes [2]: [max#68, max#69] -Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] +Aggregate Attributes [2]: [max#67, max#68] +Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#69, max#70] (65) HashAggregate [codegen id : 54] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#69, max#70] Keys [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] Functions [2]: [max(web_sales#64), max(store_sales#65)] -Aggregate Attributes [2]: [max(web_sales#64)#72, max(store_sales#65)#73] -Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max(web_sales#64)#72 AS web_cumulative#74, max(store_sales#65)#73 AS store_cumulative#75] +Aggregate Attributes [2]: [max(web_sales#64)#71, max(store_sales#65)#72] +Results [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max(web_sales#64)#71 AS web_cumulative#73, max(store_sales#65)#72 AS store_cumulative#74] (66) Filter [codegen id : 54] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#74, store_cumulative#75] -Condition : ((isnotnull(web_cumulative#74) AND isnotnull(store_cumulative#75)) AND (web_cumulative#74 > store_cumulative#75)) +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] +Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) AND (web_cumulative#73 > store_cumulative#74)) (67) TakeOrderedAndProject -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#74, store_cumulative#75] -Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#74, store_cumulative#75] +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] +Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] ===== Subqueries ===== @@ -384,26 +384,26 @@ BroadcastExchange (72) (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#76] +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#76] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#75] (70) Filter [codegen id : 1] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#76] -Condition : (((isnotnull(d_month_seq#76) AND (d_month_seq#76 >= 1212)) AND (d_month_seq#76 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#75] +Condition : (((isnotnull(d_month_seq#75) AND (d_month_seq#75 >= 1212)) AND (d_month_seq#75 <= 1223)) AND isnotnull(d_date_sk#5)) (71) Project [codegen id : 1] Output [2]: [d_date_sk#5, d_date#6] -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#76] +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#75] (72) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index e7cd713d07cb4..d214b321a4791 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -207,77 +207,77 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] Keys [5]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31] Functions [1]: [sum(UnscaledValue(cs_sales_price#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#34] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#34,17,2) AS sum_sales#35] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#33))#20] +Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#33))#20,17,2) AS sum_sales#21] (36) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#34] (37) Sort [codegen id : 21] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST], false, 0 (38) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] -Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21] +Arguments: [rank(d_year#30, d_moy#31) windowspecdefinition(i_category#27, i_brand#28, cc_name#29, d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (39) Project [codegen id : 22] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#21 AS sum_sales#36, rn#35] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#21, rn#35] (40) Exchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#37 + 1), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, (rn#35 + 1), 5), ENSURE_REQUIREMENTS, [id=#37] (41) Sort [codegen id : 23] -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#37 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] +Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#29 ASC NULLS FIRST, (rn#35 + 1) ASC NULLS FIRST], false, 0 (42) SortMergeJoin [codegen id : 24] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#35 + 1)] Join condition: None (43) Project [codegen id : 24] -Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35] -Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36] +Input [13]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, i_category#27, i_brand#28, cc_name#29, sum_sales#36, rn#35] (44) ReusedExchange [Reuses operator id: 36] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] +Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] (45) Sort [codegen id : 33] -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST], false, 0 (46) Window -Input [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -Arguments: [rank(d_year#42, d_moy#43) windowspecdefinition(i_category#39, i_brand#40, cc_name#41, d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#45], [i_category#39, i_brand#40, cc_name#41], [d_year#42 ASC NULLS FIRST, d_moy#43 ASC NULLS FIRST] +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21] +Arguments: [rank(d_year#41, d_moy#42) windowspecdefinition(i_category#38, i_brand#39, cc_name#40, d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] (47) Project [codegen id : 34] -Output [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#21 AS sum_sales#44, rn#43] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#21, rn#43] (48) Exchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: hashpartitioning(i_category#39, i_brand#40, cc_name#41, (rn#45 - 1), 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] +Arguments: hashpartitioning(i_category#38, i_brand#39, cc_name#40, (rn#43 - 1), 5), ENSURE_REQUIREMENTS, [id=#45] (49) Sort [codegen id : 35] -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: [i_category#39 ASC NULLS FIRST, i_brand#40 ASC NULLS FIRST, cc_name#41 ASC NULLS FIRST, (rn#45 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] +Arguments: [i_category#38 ASC NULLS FIRST, i_brand#39 ASC NULLS FIRST, cc_name#40 ASC NULLS FIRST, (rn#43 - 1) ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 36] Left keys [4]: [i_category#15, i_brand#14, cc_name#10, rn#24] -Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] +Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#43 - 1)] Join condition: None (51) Project [codegen id : 36] -Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, sum_sales#36 AS psum#46, sum_sales#44 AS nsum#47] +Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales#21, avg_monthly_sales#25, rn#24, sum_sales#36, i_category#38, i_brand#39, cc_name#40, sum_sales#44, rn#43] (52) TakeOrderedAndProject -Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] ===== Subqueries ===== @@ -304,6 +304,6 @@ Condition : ((((d_year#7 = 1999) OR ((d_year#7 = 1998) AND (d_moy#8 = 12))) OR ( (56) BroadcastExchange Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#48] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index 1fb68bac3c7d8..65a811671c32d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -180,69 +180,69 @@ Output [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum#30] Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum#30] Keys [5]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29] Functions [1]: [sum(UnscaledValue(cs_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#31))#32] -Results [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, MakeDecimal(sum(UnscaledValue(cs_sales_price#31))#32,17,2) AS sum_sales#33] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#31))#19] +Results [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, MakeDecimal(sum(UnscaledValue(cs_sales_price#31))#19,17,2) AS sum_sales#20] (31) Exchange -Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33] -Arguments: hashpartitioning(i_category#25, i_brand#26, cc_name#27, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20] +Arguments: hashpartitioning(i_category#25, i_brand#26, cc_name#27, 5), ENSURE_REQUIREMENTS, [id=#32] (32) Sort [codegen id : 13] -Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33] +Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20] Arguments: [i_category#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, cc_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST], false, 0 (33) Window -Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33] -Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] +Input [6]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20] +Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#25, i_brand#26, cc_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#25, i_brand#26, cc_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] (34) Project [codegen id : 14] -Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] -Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#33, rn#35] +Output [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#20 AS sum_sales#34, rn#33] +Input [7]: [i_category#25, i_brand#26, cc_name#27, d_year#28, d_moy#29, sum_sales#20, rn#33] (35) BroadcastExchange -Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#36] +Input [5]: [i_category#25, i_brand#26, cc_name#27, sum_sales#34, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#35] (36) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#35 + 1)] +Right keys [4]: [i_category#25, i_brand#26, cc_name#27, (rn#33 + 1)] Join condition: None (37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33] -Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#33, rn#35] +Output [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#34] +Input [13]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, i_category#25, i_brand#26, cc_name#27, sum_sales#34, rn#33] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] +Output [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20] (39) Sort [codegen id : 20] -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [i_category#37 ASC NULLS FIRST, i_brand#38 ASC NULLS FIRST, cc_name#39 ASC NULLS FIRST, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20] +Arguments: [i_category#36 ASC NULLS FIRST, i_brand#37 ASC NULLS FIRST, cc_name#38 ASC NULLS FIRST, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42] -Arguments: [rank(d_year#40, d_moy#41) windowspecdefinition(i_category#37, i_brand#38, cc_name#39, d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#43], [i_category#37, i_brand#38, cc_name#39], [d_year#40 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST] +Input [6]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20] +Arguments: [rank(d_year#39, d_moy#40) windowspecdefinition(i_category#36, i_brand#37, cc_name#38, d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#36, i_brand#37, cc_name#38], [d_year#39 ASC NULLS FIRST, d_moy#40 ASC NULLS FIRST] (41) Project [codegen id : 21] -Output [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Input [7]: [i_category#37, i_brand#38, cc_name#39, d_year#40, d_moy#41, sum_sales#42, rn#43] +Output [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#20 AS sum_sales#42, rn#41] +Input [7]: [i_category#36, i_brand#37, cc_name#38, d_year#39, d_moy#40, sum_sales#20, rn#41] (42) BroadcastExchange -Input [5]: [i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#44] +Input [5]: [i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#43] (43) BroadcastHashJoin [codegen id : 22] Left keys [4]: [i_category#3, i_brand#2, cc_name#14, rn#23] -Right keys [4]: [i_category#37, i_brand#38, cc_name#39, (rn#43 - 1)] +Right keys [4]: [i_category#36, i_brand#37, cc_name#38, (rn#41 - 1)] Join condition: None (44) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#33 AS psum#45, sum_sales#42 AS nsum#46] -Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#33, i_category#37, i_brand#38, cc_name#39, sum_sales#42, rn#43] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, sum_sales#34 AS psum#44, sum_sales#42 AS nsum#45] +Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales#20, avg_monthly_sales#24, rn#23, sum_sales#34, i_category#36, i_brand#37, cc_name#38, sum_sales#42, rn#41] (45) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#45, nsum#46] +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] ===== Subqueries ===== @@ -269,6 +269,6 @@ Condition : ((((d_year#11 = 1999) OR ((d_year#11 = 1998) AND (d_moy#12 = 12))) O (49) BroadcastExchange Input [3]: [d_date_sk#10, d_year#11, d_moy#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 00b1c83cf9d2e..b6a5a36a10c6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -437,60 +437,60 @@ Aggregate Attributes [3]: [sum(sales#41)#148, sum(returns#42)#149, sum(profit#43 Results [5]: [channel#39, id#40, cast(sum(sales#41)#148 as decimal(37,2)) AS sales#151, cast(sum(returns#42)#149 as decimal(37,2)) AS returns#152, cast(sum(profit#43)#150 as decimal(38,2)) AS profit#153] (76) ReusedExchange [Reuses operator id: 74] -Output [8]: [channel#39, id#40, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] +Output [8]: [channel#39, id#40, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] (77) HashAggregate [codegen id : 48] -Input [8]: [channel#39, id#40, sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] +Input [8]: [channel#39, id#40, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] Keys [2]: [channel#39, id#40] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#160, sum(returns#42)#161, sum(profit#43)#162] -Results [4]: [channel#39, sum(sales#41)#160 AS sales#163, sum(returns#42)#161 AS returns#164, sum(profit#43)#162 AS profit#165] +Aggregate Attributes [3]: [sum(sales#41)#148, sum(returns#42)#149, sum(profit#43)#150] +Results [4]: [channel#39, sum(sales#41)#148 AS sales#154, sum(returns#42)#149 AS returns#155, sum(profit#43)#150 AS profit#156] (78) HashAggregate [codegen id : 48] -Input [4]: [channel#39, sales#163, returns#164, profit#165] +Input [4]: [channel#39, sales#154, returns#155, profit#156] Keys [1]: [channel#39] -Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] -Aggregate Attributes [6]: [sum#166, isEmpty#167, sum#168, isEmpty#169, sum#170, isEmpty#171] -Results [7]: [channel#39, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Functions [3]: [partial_sum(sales#154), partial_sum(returns#155), partial_sum(profit#156)] +Aggregate Attributes [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Results [7]: [channel#39, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] (79) Exchange -Input [7]: [channel#39, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -Arguments: hashpartitioning(channel#39, 5), ENSURE_REQUIREMENTS, [id=#178] +Input [7]: [channel#39, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +Arguments: hashpartitioning(channel#39, 5), ENSURE_REQUIREMENTS, [id=#169] (80) HashAggregate [codegen id : 49] -Input [7]: [channel#39, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Input [7]: [channel#39, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] Keys [1]: [channel#39] -Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] -Aggregate Attributes [3]: [sum(sales#163)#179, sum(returns#164)#180, sum(profit#165)#181] -Results [5]: [channel#39, null AS id#182, sum(sales#163)#179 AS sum(sales)#183, sum(returns#164)#180 AS sum(returns)#184, sum(profit#165)#181 AS sum(profit)#185] +Functions [3]: [sum(sales#154), sum(returns#155), sum(profit#156)] +Aggregate Attributes [3]: [sum(sales#154)#170, sum(returns#155)#171, sum(profit#156)#172] +Results [5]: [channel#39, null AS id#173, sum(sales#154)#170 AS sum(sales)#174, sum(returns#155)#171 AS sum(returns)#175, sum(profit#156)#172 AS sum(profit)#176] (81) ReusedExchange [Reuses operator id: 74] -Output [8]: [channel#39, id#40, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Output [8]: [channel#39, id#40, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] (82) HashAggregate [codegen id : 73] -Input [8]: [channel#39, id#40, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Input [8]: [channel#39, id#40, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] Keys [2]: [channel#39, id#40] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#192, sum(returns#42)#193, sum(profit#43)#194] -Results [3]: [sum(sales#41)#192 AS sales#163, sum(returns#42)#193 AS returns#164, sum(profit#43)#194 AS profit#165] +Aggregate Attributes [3]: [sum(sales#41)#148, sum(returns#42)#149, sum(profit#43)#150] +Results [3]: [sum(sales#41)#148 AS sales#154, sum(returns#42)#149 AS returns#155, sum(profit#43)#150 AS profit#156] (83) HashAggregate [codegen id : 73] -Input [3]: [sales#163, returns#164, profit#165] +Input [3]: [sales#154, returns#155, profit#156] Keys: [] -Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] -Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] -Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Functions [3]: [partial_sum(sales#154), partial_sum(returns#155), partial_sum(profit#156)] +Aggregate Attributes [6]: [sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182] +Results [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] (84) Exchange -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#189] (85) HashAggregate [codegen id : 74] -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] Keys: [] -Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] -Aggregate Attributes [3]: [sum(sales#163)#208, sum(returns#164)#209, sum(profit#165)#210] -Results [5]: [null AS channel#211, null AS id#212, sum(sales#163)#208 AS sum(sales)#213, sum(returns#164)#209 AS sum(returns)#214, sum(profit#165)#210 AS sum(profit)#215] +Functions [3]: [sum(sales#154), sum(returns#155), sum(profit#156)] +Aggregate Attributes [3]: [sum(sales#154)#190, sum(returns#155)#191, sum(profit#156)#192] +Results [5]: [null AS channel#193, null AS id#194, sum(sales#154)#190 AS sum(sales)#195, sum(returns#155)#191 AS sum(returns)#196, sum(profit#156)#192 AS sum(profit)#197] (86) Union @@ -503,7 +503,7 @@ Results [5]: [channel#39, id#40, sales#151, returns#152, profit#153] (88) Exchange Input [5]: [channel#39, id#40, sales#151, returns#152, profit#153] -Arguments: hashpartitioning(channel#39, id#40, sales#151, returns#152, profit#153, 5), ENSURE_REQUIREMENTS, [id=#216] +Arguments: hashpartitioning(channel#39, id#40, sales#151, returns#152, profit#153, 5), ENSURE_REQUIREMENTS, [id=#198] (89) HashAggregate [codegen id : 76] Input [5]: [channel#39, id#40, sales#151, returns#152, profit#153] @@ -527,26 +527,26 @@ BroadcastExchange (95) (91) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_date#217] +Output [2]: [d_date_sk#25, d_date#199] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_date#217] +Input [2]: [d_date_sk#25, d_date#199] (93) Filter [codegen id : 1] -Input [2]: [d_date_sk#25, d_date#217] -Condition : (((isnotnull(d_date#217) AND (d_date#217 >= 1998-08-04)) AND (d_date#217 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#25, d_date#199] +Condition : (((isnotnull(d_date#199) AND (d_date#199 >= 1998-08-04)) AND (d_date#199 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) (94) Project [codegen id : 1] Output [1]: [d_date_sk#25] -Input [2]: [d_date_sk#25, d_date#217] +Input [2]: [d_date_sk#25, d_date#199] (95) BroadcastExchange Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#218] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#200] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 8b44187cfe1c1..05636f5f44067 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -422,60 +422,60 @@ Aggregate Attributes [3]: [sum(sales#41)#147, sum(returns#42)#148, sum(profit#43 Results [5]: [channel#39, id#40, cast(sum(sales#41)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#42)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#43)#149 as decimal(38,2)) AS profit#152] (73) ReusedExchange [Reuses operator id: 71] -Output [8]: [channel#39, id#40, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Output [8]: [channel#39, id#40, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] (74) HashAggregate [codegen id : 42] -Input [8]: [channel#39, id#40, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Input [8]: [channel#39, id#40, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#39, id#40] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#159, sum(returns#42)#160, sum(profit#43)#161] -Results [4]: [channel#39, sum(sales#41)#159 AS sales#162, sum(returns#42)#160 AS returns#163, sum(profit#43)#161 AS profit#164] +Aggregate Attributes [3]: [sum(sales#41)#147, sum(returns#42)#148, sum(profit#43)#149] +Results [4]: [channel#39, sum(sales#41)#147 AS sales#153, sum(returns#42)#148 AS returns#154, sum(profit#43)#149 AS profit#155] (75) HashAggregate [codegen id : 42] -Input [4]: [channel#39, sales#162, returns#163, profit#164] +Input [4]: [channel#39, sales#153, returns#154, profit#155] Keys [1]: [channel#39] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Results [7]: [channel#39, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Results [7]: [channel#39, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (76) Exchange -Input [7]: [channel#39, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: hashpartitioning(channel#39, 5), ENSURE_REQUIREMENTS, [id=#177] +Input [7]: [channel#39, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#39, 5), ENSURE_REQUIREMENTS, [id=#168] (77) HashAggregate [codegen id : 43] -Input [7]: [channel#39, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [7]: [channel#39, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#39] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#178, sum(returns#163)#179, sum(profit#164)#180] -Results [5]: [channel#39, null AS id#181, sum(sales#162)#178 AS sum(sales)#182, sum(returns#163)#179 AS sum(returns)#183, sum(profit#164)#180 AS sum(profit)#184] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#169, sum(returns#154)#170, sum(profit#155)#171] +Results [5]: [channel#39, null AS id#172, sum(sales#153)#169 AS sum(sales)#173, sum(returns#154)#170 AS sum(returns)#174, sum(profit#155)#171 AS sum(profit)#175] (78) ReusedExchange [Reuses operator id: 71] -Output [8]: [channel#39, id#40, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Output [8]: [channel#39, id#40, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] (79) HashAggregate [codegen id : 64] -Input [8]: [channel#39, id#40, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [8]: [channel#39, id#40, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#39, id#40] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#191, sum(returns#42)#192, sum(profit#43)#193] -Results [3]: [sum(sales#41)#191 AS sales#162, sum(returns#42)#192 AS returns#163, sum(profit#43)#193 AS profit#164] +Aggregate Attributes [3]: [sum(sales#41)#147, sum(returns#42)#148, sum(profit#43)#149] +Results [3]: [sum(sales#41)#147 AS sales#153, sum(returns#42)#148 AS returns#154, sum(profit#43)#149 AS profit#155] (80) HashAggregate [codegen id : 64] -Input [3]: [sales#162, returns#163, profit#164] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -Aggregate Attributes [6]: [sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199] -Results [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Results [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] (81) Exchange -Input [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#206] +Input [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#188] (82) HashAggregate [codegen id : 65] -Input [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Input [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] Keys: [] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#207, sum(returns#163)#208, sum(profit#164)#209] -Results [5]: [null AS channel#210, null AS id#211, sum(sales#162)#207 AS sum(sales)#212, sum(returns#163)#208 AS sum(returns)#213, sum(profit#164)#209 AS sum(profit)#214] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#189, sum(returns#154)#190, sum(profit#155)#191] +Results [5]: [null AS channel#192, null AS id#193, sum(sales#153)#189 AS sum(sales)#194, sum(returns#154)#190 AS sum(returns)#195, sum(profit#155)#191 AS sum(profit)#196] (83) Union @@ -488,7 +488,7 @@ Results [5]: [channel#39, id#40, sales#150, returns#151, profit#152] (85) Exchange Input [5]: [channel#39, id#40, sales#150, returns#151, profit#152] -Arguments: hashpartitioning(channel#39, id#40, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#215] +Arguments: hashpartitioning(channel#39, id#40, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#197] (86) HashAggregate [codegen id : 67] Input [5]: [channel#39, id#40, sales#150, returns#151, profit#152] @@ -512,26 +512,26 @@ BroadcastExchange (92) (88) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#216] +Output [2]: [d_date_sk#22, d_date#198] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct (89) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#216] +Input [2]: [d_date_sk#22, d_date#198] (90) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#216] -Condition : (((isnotnull(d_date#216) AND (d_date#216 >= 1998-08-04)) AND (d_date#216 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#198] +Condition : (((isnotnull(d_date#198) AND (d_date#198 >= 1998-08-04)) AND (d_date#198 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (91) Project [codegen id : 1] Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#216] +Input [2]: [d_date_sk#22, d_date#198] (92) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#217] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#199] Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 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 393a4211487ad..19240a79cc91c 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 @@ -1068,44 +1068,44 @@ Input [19]: [ss_item_sk#128, ss_wholesale_cost#136, ss_list_price#137, ss_coupon Input [18]: [ss_wholesale_cost#136, ss_list_price#137, ss_coupon_amt#138, d_year#146, d_year#158, d_year#160, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, i_item_sk#186, i_product_name#187] Keys [15]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#136)), partial_sum(UnscaledValue(ss_list_price#137)), partial_sum(UnscaledValue(ss_coupon_amt#138))] -Aggregate Attributes [4]: [count#188, sum#189, sum#190, sum#191] -Results [19]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, count#192, sum#193, sum#194, sum#195] +Aggregate Attributes [4]: [count#97, sum#188, sum#189, sum#190] +Results [19]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, count#101, sum#191, sum#192, sum#193] (202) Exchange -Input [19]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, count#192, sum#193, sum#194, sum#195] -Arguments: hashpartitioning(i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#196] +Input [19]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, count#101, sum#191, sum#192, sum#193] +Arguments: hashpartitioning(i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#194] (203) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, count#192, sum#193, sum#194, sum#195] +Input [19]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160, count#101, sum#191, sum#192, sum#193] Keys [15]: [i_product_name#187, i_item_sk#186, s_store_name#148, s_zip#149, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#180, ca_street_name#181, ca_city#182, ca_zip#183, d_year#146, d_year#158, d_year#160] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#136)), sum(UnscaledValue(ss_list_price#137)), sum(UnscaledValue(ss_coupon_amt#138))] -Aggregate Attributes [4]: [count(1)#197, sum(UnscaledValue(ss_wholesale_cost#136))#198, sum(UnscaledValue(ss_list_price#137))#199, sum(UnscaledValue(ss_coupon_amt#138))#200] -Results [8]: [i_item_sk#186 AS item_sk#201, s_store_name#148 AS store_name#202, s_zip#149 AS store_zip#203, d_year#146 AS syear#204, count(1)#197 AS cnt#205, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#136))#198,17,2) AS s1#206, MakeDecimal(sum(UnscaledValue(ss_list_price#137))#199,17,2) AS s2#207, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#138))#200,17,2) AS s3#208] +Aggregate Attributes [4]: [count(1)#106, sum(UnscaledValue(ss_wholesale_cost#136))#107, sum(UnscaledValue(ss_list_price#137))#108, sum(UnscaledValue(ss_coupon_amt#138))#109] +Results [8]: [i_item_sk#186 AS item_sk#195, s_store_name#148 AS store_name#196, s_zip#149 AS store_zip#197, d_year#146 AS syear#198, count(1)#106 AS cnt#199, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#136))#107,17,2) AS s1#200, MakeDecimal(sum(UnscaledValue(ss_list_price#137))#108,17,2) AS s2#201, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#138))#109,17,2) AS s3#202] (204) Exchange -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] +Input [8]: [item_sk#195, store_name#196, store_zip#197, syear#198, cnt#199, s1#200, s2#201, s3#202] +Arguments: hashpartitioning(item_sk#195, store_name#196, store_zip#197, 5), ENSURE_REQUIREMENTS, [id=#203] (205) Sort [codegen id : 86] -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 +Input [8]: [item_sk#195, store_name#196, store_zip#197, syear#198, cnt#199, s1#200, s2#201, s3#202] +Arguments: [item_sk#195 ASC NULLS FIRST, store_name#196 ASC NULLS FIRST, store_zip#197 ASC NULLS FIRST], false, 0 (206) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#111, store_name#112, store_zip#113] -Right keys [3]: [item_sk#201, store_name#202, store_zip#203] -Join condition: (cnt#205 <= cnt#123) +Right keys [3]: [item_sk#195, store_name#196, store_zip#197] +Join condition: (cnt#199 <= cnt#123) (207) Project [codegen id : 87] -Output [21]: [product_name#110, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, s1#206, s2#207, s3#208, syear#204, cnt#205] -Input [25]: [product_name#110, item_sk#111, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, item_sk#201, store_name#202, store_zip#203, syear#204, cnt#205, s1#206, s2#207, s3#208] +Output [21]: [product_name#110, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, s1#200, s2#201, s3#202, syear#198, cnt#199] +Input [25]: [product_name#110, item_sk#111, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, item_sk#195, store_name#196, store_zip#197, syear#198, cnt#199, s1#200, s2#201, s3#202] (208) Exchange -Input [21]: [product_name#110, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, s1#206, s2#207, s3#208, syear#204, cnt#205] -Arguments: rangepartitioning(product_name#110 ASC NULLS FIRST, store_name#112 ASC NULLS FIRST, cnt#205 ASC NULLS FIRST, s1#124 ASC NULLS FIRST, s1#206 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#210] +Input [21]: [product_name#110, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, s1#200, s2#201, s3#202, syear#198, cnt#199] +Arguments: rangepartitioning(product_name#110 ASC NULLS FIRST, store_name#112 ASC NULLS FIRST, cnt#199 ASC NULLS FIRST, s1#124 ASC NULLS FIRST, s1#200 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#204] (209) Sort [codegen id : 88] -Input [21]: [product_name#110, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, s1#206, s2#207, s3#208, syear#204, cnt#205] -Arguments: [product_name#110 ASC NULLS FIRST, store_name#112 ASC NULLS FIRST, cnt#205 ASC NULLS FIRST, s1#124 ASC NULLS FIRST, s1#206 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#110, store_name#112, store_zip#113, b_street_number#114, b_streen_name#115, b_city#116, b_zip#117, c_street_number#118, c_street_name#119, c_city#120, c_zip#121, syear#122, cnt#123, s1#124, s2#125, s3#126, s1#200, s2#201, s3#202, syear#198, cnt#199] +Arguments: [product_name#110 ASC NULLS FIRST, store_name#112 ASC NULLS FIRST, cnt#199 ASC NULLS FIRST, s1#124 ASC NULLS FIRST, s1#200 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1132,7 +1132,7 @@ Condition : ((isnotnull(d_year#44) AND (d_year#44 = 1999)) AND isnotnull(d_date_ (213) BroadcastExchange Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#211] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#205] Subquery:2 Hosting operator id = 129 Hosting Expression = ss_sold_date_sk#139 IN dynamicpruning#140 BroadcastExchange (217) @@ -1157,6 +1157,6 @@ Condition : ((isnotnull(d_year#146) AND (d_year#146 = 2000)) AND isnotnull(d_dat (217) BroadcastExchange Input [2]: [d_date_sk#145, d_year#146] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#212] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#206] 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 07754b2d517a4..ddaa34ab4e657 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 @@ -740,8 +740,8 @@ Output [4]: [cs_item_sk#139, sum#140, sum#141, isEmpty#142] Input [4]: [cs_item_sk#139, sum#140, sum#141, isEmpty#142] Keys [1]: [cs_item_sk#139] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#143)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#143))#147, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#148] -Results [3]: [cs_item_sk#139, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#143))#147,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#148 AS refund#41] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#143))#38, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#39] +Results [3]: [cs_item_sk#139, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#143))#38,17,2) AS sale#40, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#144 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#145 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#146 as decimal(9,2)))), DecimalType(9,2), true))#39 AS refund#41] (126) Filter [codegen id : 35] Input [3]: [cs_item_sk#139, sale#40, refund#41] @@ -765,223 +765,223 @@ Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#1 Input [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, cs_item_sk#139] (131) ReusedExchange [Reuses operator id: 191] -Output [2]: [d_date_sk#149, d_year#150] +Output [2]: [d_date_sk#147, d_year#148] (132) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_sold_date_sk#132] -Right keys [1]: [d_date_sk#149] +Right keys [1]: [d_date_sk#147] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150] -Input [13]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, d_date_sk#149, d_year#150] +Output [11]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148] +Input [13]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, ss_sold_date_sk#132, d_date_sk#147, d_year#148] (134) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#151, s_store_name#152, s_zip#153] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (135) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_store_sk#126] -Right keys [1]: [s_store_sk#151] +Right keys [1]: [s_store_sk#149] Join condition: None (136) Project [codegen id : 51] -Output [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153] -Input [14]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_sk#151, s_store_name#152, s_zip#153] +Output [12]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_store_sk#126, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_sk#149, s_store_name#150, s_zip#151] (137) ReusedExchange [Reuses operator id: 47] -Output [6]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, c_first_shipto_date_sk#158, c_first_sales_date_sk#159] +Output [6]: [c_customer_sk#152, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157] (138) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_customer_sk#122] -Right keys [1]: [c_customer_sk#154] +Right keys [1]: [c_customer_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, c_first_shipto_date_sk#158, c_first_sales_date_sk#159] -Input [18]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, c_first_shipto_date_sk#158, c_first_sales_date_sk#159] +Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157] +Input [18]: [ss_item_sk#121, ss_customer_sk#122, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_customer_sk#152, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157] (140) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#160, d_year#161] +Output [2]: [d_date_sk#158, d_year#159] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#159] -Right keys [1]: [d_date_sk#160] +Left keys [1]: [c_first_sales_date_sk#157] +Right keys [1]: [d_date_sk#158] Join condition: None (142) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, c_first_shipto_date_sk#158, d_year#161] -Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, c_first_shipto_date_sk#158, c_first_sales_date_sk#159, d_date_sk#160, d_year#161] +Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, d_year#159] +Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, c_first_sales_date_sk#157, d_date_sk#158, d_year#159] (143) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#162, d_year#163] +Output [2]: [d_date_sk#160, d_year#161] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#158] -Right keys [1]: [d_date_sk#162] +Left keys [1]: [c_first_shipto_date_sk#156] +Right keys [1]: [d_date_sk#160] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163] -Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, c_first_shipto_date_sk#158, d_year#161, d_date_sk#162, d_year#163] +Output [16]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161] +Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, c_first_shipto_date_sk#156, d_year#159, d_date_sk#160, d_year#161] (146) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#164, cd_marital_status#165] +Output [2]: [cd_demo_sk#162, cd_marital_status#163] (147) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_cdemo_sk#123] -Right keys [1]: [cd_demo_sk#164] +Right keys [1]: [cd_demo_sk#162] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, cd_marital_status#165] -Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, cd_demo_sk#164, cd_marital_status#165] +Output [16]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_marital_status#163] +Input [18]: [ss_item_sk#121, ss_cdemo_sk#123, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_demo_sk#162, cd_marital_status#163] (149) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#166, cd_marital_status#167] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#155] -Right keys [1]: [cd_demo_sk#166] -Join condition: NOT (cd_marital_status#165 = cd_marital_status#167) +Left keys [1]: [c_current_cdemo_sk#153] +Right keys [1]: [cd_demo_sk#164] +Join condition: NOT (cd_marital_status#163 = cd_marital_status#165) (151) Project [codegen id : 51] -Output [14]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163] -Input [18]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_cdemo_sk#155, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, cd_marital_status#165, cd_demo_sk#166, cd_marital_status#167] +Output [14]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161] +Input [18]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_cdemo_sk#153, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, cd_marital_status#163, cd_demo_sk#164, cd_marital_status#165] (152) ReusedExchange [Reuses operator id: 71] -Output [1]: [p_promo_sk#168] +Output [1]: [p_promo_sk#166] (153) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_promo_sk#127] -Right keys [1]: [p_promo_sk#168] +Right keys [1]: [p_promo_sk#166] Join condition: None (154) Project [codegen id : 51] -Output [13]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163] -Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, p_promo_sk#168] +Output [13]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161] +Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_promo_sk#127, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, p_promo_sk#166] (155) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#169, hd_income_band_sk#170] +Output [2]: [hd_demo_sk#167, hd_income_band_sk#168] (156) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_hdemo_sk#124] -Right keys [1]: [hd_demo_sk#169] +Right keys [1]: [hd_demo_sk#167] Join condition: None (157) Project [codegen id : 51] -Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, hd_income_band_sk#170] -Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, hd_demo_sk#169, hd_income_band_sk#170] +Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168] +Input [15]: [ss_item_sk#121, ss_hdemo_sk#124, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_demo_sk#167, hd_income_band_sk#168] (158) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#171, hd_income_band_sk#172] +Output [2]: [hd_demo_sk#169, hd_income_band_sk#170] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#156] -Right keys [1]: [hd_demo_sk#171] +Left keys [1]: [c_current_hdemo_sk#154] +Right keys [1]: [hd_demo_sk#169] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_addr_sk#157, d_year#161, d_year#163, hd_income_band_sk#170, hd_income_band_sk#172] -Input [15]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_hdemo_sk#156, c_current_addr_sk#157, d_year#161, d_year#163, hd_income_band_sk#170, hd_demo_sk#171, hd_income_band_sk#172] +Output [13]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170] +Input [15]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_hdemo_sk#154, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_demo_sk#169, hd_income_band_sk#170] (161) ReusedExchange [Reuses operator id: 86] -Output [5]: [ca_address_sk#173, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177] +Output [5]: [ca_address_sk#171, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175] (162) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_addr_sk#125] -Right keys [1]: [ca_address_sk#173] +Right keys [1]: [ca_address_sk#171] Join condition: None (163) Project [codegen id : 51] -Output [16]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_addr_sk#157, d_year#161, d_year#163, hd_income_band_sk#170, hd_income_band_sk#172, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177] -Input [18]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_addr_sk#157, d_year#161, d_year#163, hd_income_band_sk#170, hd_income_band_sk#172, ca_address_sk#173, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177] +Output [16]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175] +Input [18]: [ss_item_sk#121, ss_addr_sk#125, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_address_sk#171, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175] (164) ReusedExchange [Reuses operator id: 86] -Output [5]: [ca_address_sk#178, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182] +Output [5]: [ca_address_sk#176, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#157] -Right keys [1]: [ca_address_sk#178] +Left keys [1]: [c_current_addr_sk#155] +Right keys [1]: [ca_address_sk#176] Join condition: None (166) Project [codegen id : 51] -Output [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, d_year#161, d_year#163, hd_income_band_sk#170, hd_income_band_sk#172, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182] -Input [21]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, c_current_addr_sk#157, d_year#161, d_year#163, hd_income_band_sk#170, hd_income_band_sk#172, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_address_sk#178, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182] +Output [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] +Input [21]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, c_current_addr_sk#155, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_address_sk#176, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] (167) ReusedExchange [Reuses operator id: 95] -Output [1]: [ib_income_band_sk#183] +Output [1]: [ib_income_band_sk#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#170] -Right keys [1]: [ib_income_band_sk#183] +Left keys [1]: [hd_income_band_sk#168] +Right keys [1]: [ib_income_band_sk#181] Join condition: None (169) Project [codegen id : 51] -Output [18]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, d_year#161, d_year#163, hd_income_band_sk#172, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182] -Input [20]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, d_year#161, d_year#163, hd_income_band_sk#170, hd_income_band_sk#172, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, ib_income_band_sk#183] +Output [18]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] +Input [20]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#168, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, ib_income_band_sk#181] (170) ReusedExchange [Reuses operator id: 95] -Output [1]: [ib_income_band_sk#184] +Output [1]: [ib_income_band_sk#182] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#172] -Right keys [1]: [ib_income_band_sk#184] +Left keys [1]: [hd_income_band_sk#170] +Right keys [1]: [ib_income_band_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [17]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, d_year#161, d_year#163, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182] -Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, d_year#161, d_year#163, hd_income_band_sk#172, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, ib_income_band_sk#184] +Output [17]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180] +Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, hd_income_band_sk#170, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, ib_income_band_sk#182] (173) ReusedExchange [Reuses operator id: 105] -Output [2]: [i_item_sk#185, i_product_name#186] +Output [2]: [i_item_sk#183, i_product_name#184] (174) BroadcastHashJoin [codegen id : 51] Left keys [1]: [ss_item_sk#121] -Right keys [1]: [i_item_sk#185] +Right keys [1]: [i_item_sk#183] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, d_year#161, d_year#163, s_store_name#152, s_zip#153, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, i_item_sk#185, i_product_name#186] -Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, s_store_name#152, s_zip#153, d_year#161, d_year#163, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, i_item_sk#185, i_product_name#186] +Output [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, d_year#159, d_year#161, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184] +Input [19]: [ss_item_sk#121, ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, s_store_name#150, s_zip#151, d_year#159, d_year#161, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184] (176) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#150, d_year#161, d_year#163, s_store_name#152, s_zip#153, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, i_item_sk#185, i_product_name#186] -Keys [15]: [i_product_name#186, i_item_sk#185, s_store_name#152, s_zip#153, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, d_year#150, d_year#161, d_year#163] +Input [18]: [ss_wholesale_cost#129, ss_list_price#130, ss_coupon_amt#131, d_year#148, d_year#159, d_year#161, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, i_item_sk#183, i_product_name#184] +Keys [15]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#129)), partial_sum(UnscaledValue(ss_list_price#130)), partial_sum(UnscaledValue(ss_coupon_amt#131))] -Aggregate Attributes [4]: [count#187, sum#188, sum#189, sum#190] -Results [19]: [i_product_name#186, i_item_sk#185, s_store_name#152, s_zip#153, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, d_year#150, d_year#161, d_year#163, count#191, sum#192, sum#193, sum#194] +Aggregate Attributes [4]: [count#91, sum#185, sum#186, sum#187] +Results [19]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161, count#95, sum#188, sum#189, sum#190] (177) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#186, i_item_sk#185, s_store_name#152, s_zip#153, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, d_year#150, d_year#161, d_year#163, count#191, sum#192, sum#193, sum#194] -Keys [15]: [i_product_name#186, i_item_sk#185, s_store_name#152, s_zip#153, ca_street_number#174, ca_street_name#175, ca_city#176, ca_zip#177, ca_street_number#179, ca_street_name#180, ca_city#181, ca_zip#182, d_year#150, d_year#161, d_year#163] +Input [19]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161, count#95, sum#188, sum#189, sum#190] +Keys [15]: [i_product_name#184, i_item_sk#183, s_store_name#150, s_zip#151, ca_street_number#172, ca_street_name#173, ca_city#174, ca_zip#175, ca_street_number#177, ca_street_name#178, ca_city#179, ca_zip#180, d_year#148, d_year#159, d_year#161] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#129)), sum(UnscaledValue(ss_list_price#130)), sum(UnscaledValue(ss_coupon_amt#131))] -Aggregate Attributes [4]: [count(1)#195, sum(UnscaledValue(ss_wholesale_cost#129))#196, sum(UnscaledValue(ss_list_price#130))#197, sum(UnscaledValue(ss_coupon_amt#131))#198] -Results [8]: [i_item_sk#185 AS item_sk#199, s_store_name#152 AS store_name#200, s_zip#153 AS store_zip#201, d_year#150 AS syear#202, count(1)#195 AS cnt#203, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#129))#196,17,2) AS s1#204, MakeDecimal(sum(UnscaledValue(ss_list_price#130))#197,17,2) AS s2#205, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#131))#198,17,2) AS s3#206] +Aggregate Attributes [4]: [count(1)#99, sum(UnscaledValue(ss_wholesale_cost#129))#100, sum(UnscaledValue(ss_list_price#130))#101, sum(UnscaledValue(ss_coupon_amt#131))#102] +Results [8]: [i_item_sk#183 AS item_sk#191, s_store_name#150 AS store_name#192, s_zip#151 AS store_zip#193, d_year#148 AS syear#194, count(1)#99 AS cnt#195, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#129))#100,17,2) AS s1#196, MakeDecimal(sum(UnscaledValue(ss_list_price#130))#101,17,2) AS s2#197, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#131))#102,17,2) AS s3#198] (178) Exchange -Input [8]: [item_sk#199, store_name#200, store_zip#201, syear#202, cnt#203, s1#204, s2#205, s3#206] -Arguments: hashpartitioning(item_sk#199, store_name#200, store_zip#201, 5), ENSURE_REQUIREMENTS, [id=#207] +Input [8]: [item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198] +Arguments: hashpartitioning(item_sk#191, store_name#192, store_zip#193, 5), ENSURE_REQUIREMENTS, [id=#199] (179) Sort [codegen id : 52] -Input [8]: [item_sk#199, store_name#200, store_zip#201, syear#202, cnt#203, s1#204, s2#205, s3#206] -Arguments: [item_sk#199 ASC NULLS FIRST, store_name#200 ASC NULLS FIRST, store_zip#201 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198] +Arguments: [item_sk#191 ASC NULLS FIRST, store_name#192 ASC NULLS FIRST, store_zip#193 ASC NULLS FIRST], false, 0 (180) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#104, store_name#105, store_zip#106] -Right keys [3]: [item_sk#199, store_name#200, store_zip#201] -Join condition: (cnt#203 <= cnt#116) +Right keys [3]: [item_sk#191, store_name#192, store_zip#193] +Join condition: (cnt#195 <= cnt#116) (181) Project [codegen id : 53] -Output [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#204, s2#205, s3#206, syear#202, cnt#203] -Input [25]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, item_sk#199, store_name#200, store_zip#201, syear#202, cnt#203, s1#204, s2#205, s3#206] +Output [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195] +Input [25]: [product_name#103, item_sk#104, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, item_sk#191, store_name#192, store_zip#193, syear#194, cnt#195, s1#196, s2#197, s3#198] (182) Exchange -Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#204, s2#205, s3#206, syear#202, cnt#203] -Arguments: rangepartitioning(product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#203 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#204 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#208] +Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195] +Arguments: rangepartitioning(product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#195 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#196 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#200] (183) Sort [codegen id : 54] -Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#204, s2#205, s3#206, syear#202, cnt#203] -Arguments: [product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#203 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#204 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#103, store_name#105, store_zip#106, b_street_number#107, b_streen_name#108, b_city#109, b_zip#110, c_street_number#111, c_street_name#112, c_city#113, c_zip#114, syear#115, cnt#116, s1#117, s2#118, s3#119, s1#196, s2#197, s3#198, syear#194, cnt#195] +Arguments: [product_name#103 ASC NULLS FIRST, store_name#105 ASC NULLS FIRST, cnt#195 ASC NULLS FIRST, s1#117 ASC NULLS FIRST, s1#196 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1008,7 +1008,7 @@ Condition : ((isnotnull(d_year#43) AND (d_year#43 = 1999)) AND isnotnull(d_date_ (187) BroadcastExchange Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#209] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#201] Subquery:2 Hosting operator id = 112 Hosting Expression = ss_sold_date_sk#132 IN dynamicpruning#133 BroadcastExchange (191) @@ -1018,21 +1018,21 @@ BroadcastExchange (191) (188) Scan parquet default.date_dim -Output [2]: [d_date_sk#149, d_year#150] +Output [2]: [d_date_sk#147, d_year#148] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (189) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#149, d_year#150] +Input [2]: [d_date_sk#147, d_year#148] (190) Filter [codegen id : 1] -Input [2]: [d_date_sk#149, d_year#150] -Condition : ((isnotnull(d_year#150) AND (d_year#150 = 2000)) AND isnotnull(d_date_sk#149)) +Input [2]: [d_date_sk#147, d_year#148] +Condition : ((isnotnull(d_year#148) AND (d_year#148 = 2000)) AND isnotnull(d_date_sk#147)) (191) BroadcastExchange -Input [2]: [d_date_sk#149, d_year#150] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#210] +Input [2]: [d_date_sk#147, d_year#148] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#202] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index af1fa387d16d7..b0ecc08ff8b25 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -183,234 +183,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 as decimal(38,2)) AS sumsales#27] (25) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sum#36, isEmpty#37] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#28, isEmpty#29] (26) HashAggregate [codegen id : 16] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sum#36, isEmpty#37] -Keys [8]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#38 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#39 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#38 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#39 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#40] -Results [8]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#38 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#39 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#40 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#28, isEmpty#29] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (27) HashAggregate [codegen id : 16] -Input [8]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, sumsales#41] -Keys [7]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, sum#44, isEmpty#45] +Input [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, sumsales#30] +Keys [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, sum#33, isEmpty#34] (28) Exchange -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, sum#33, isEmpty#34] +Arguments: hashpartitioning(i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, [id=#35] (29) HashAggregate [codegen id : 17] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, sum#44, isEmpty#45] -Keys [7]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#47] -Results [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, null AS s_store_id#48, sum(sumsales#41)#47 AS sumsales#49] +Input [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, sum#33, isEmpty#34] +Keys [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#36] +Results [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, null AS s_store_id#37, sum(sumsales#30)#36 AS sumsales#38] (30) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, d_moy#56, s_store_id#57, sum#58, isEmpty#59] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#39, isEmpty#40] (31) HashAggregate [codegen id : 25] -Input [10]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, d_moy#56, s_store_id#57, sum#58, isEmpty#59] -Keys [8]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, d_moy#56, s_store_id#57] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#60 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#61 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#60 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#61 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#62] -Results [7]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#60 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#61 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#62 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#39, isEmpty#40] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (32) HashAggregate [codegen id : 25] -Input [7]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, sumsales#41] -Keys [6]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [8]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, sum#65, isEmpty#66] +Input [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, sumsales#30] +Keys [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, sum#43, isEmpty#44] (33) Exchange -Input [8]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, sum#65, isEmpty#66] -Arguments: hashpartitioning(i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, sum#43, isEmpty#44] +Arguments: hashpartitioning(i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, [id=#45] (34) HashAggregate [codegen id : 26] -Input [8]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, sum#65, isEmpty#66] -Keys [6]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#68] -Results [9]: [i_category#50, i_class#51, i_brand#52, i_product_name#53, d_year#54, d_qoy#55, null AS d_moy#69, null AS s_store_id#70, sum(sumsales#41)#68 AS sumsales#71] +Input [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, sum#43, isEmpty#44] +Keys [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#46] +Results [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, null AS d_moy#47, null AS s_store_id#48, sum(sumsales#30)#46 AS sumsales#49] (35) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, d_qoy#77, d_moy#78, s_store_id#79, sum#80, isEmpty#81] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#50, isEmpty#51] (36) HashAggregate [codegen id : 34] -Input [10]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, d_qoy#77, d_moy#78, s_store_id#79, sum#80, isEmpty#81] -Keys [8]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, d_qoy#77, d_moy#78, s_store_id#79] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#82 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#83 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#82 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#83 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#84] -Results [6]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#82 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#83 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#84 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#50, isEmpty#51] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (37) HashAggregate [codegen id : 34] -Input [6]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, sumsales#41] -Keys [5]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#85, isEmpty#86] -Results [7]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, sum#87, isEmpty#88] +Input [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, sumsales#30] +Keys [5]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, sum#54, isEmpty#55] (38) Exchange -Input [7]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, sum#87, isEmpty#88] -Arguments: hashpartitioning(i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, 5), ENSURE_REQUIREMENTS, [id=#56] (39) HashAggregate [codegen id : 35] -Input [7]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, sum#87, isEmpty#88] -Keys [5]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#90] -Results [9]: [i_category#72, i_class#73, i_brand#74, i_product_name#75, d_year#76, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#41)#90 AS sumsales#94] +Input [7]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, sum#54, isEmpty#55] +Keys [5]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#57] +Results [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, null AS d_qoy#58, null AS d_moy#59, null AS s_store_id#60, sum(sumsales#30)#57 AS sumsales#61] (40) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#62, isEmpty#63] (41) HashAggregate [codegen id : 43] -Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] -Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#105 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#106 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#105 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#106 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#107] -Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#105 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#106 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#107 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#62, isEmpty#63] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [5]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (42) HashAggregate [codegen id : 43] -Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#41] -Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#108, isEmpty#109] -Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Input [5]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, sumsales#30] +Keys [4]: [i_category#18, i_class#17, i_brand#16, i_product_name#19] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#64, isEmpty#65] +Results [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, sum#66, isEmpty#67] (43) Exchange -Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] -Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [id=#112] +Input [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, sum#66, isEmpty#67] +Arguments: hashpartitioning(i_category#18, i_class#17, i_brand#16, i_product_name#19, 5), ENSURE_REQUIREMENTS, [id=#68] (44) HashAggregate [codegen id : 44] -Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] -Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#113] -Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#114, null AS d_qoy#115, null AS d_moy#116, null AS s_store_id#117, sum(sumsales#41)#113 AS sumsales#118] +Input [6]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, sum#66, isEmpty#67] +Keys [4]: [i_category#18, i_class#17, i_brand#16, i_product_name#19] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#69] +Results [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, null AS d_year#70, null AS d_qoy#71, null AS d_moy#72, null AS s_store_id#73, sum(sumsales#30)#69 AS sumsales#74] (45) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sum#127, isEmpty#128] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#75, isEmpty#76] (46) HashAggregate [codegen id : 52] -Input [10]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sum#127, isEmpty#128] -Keys [8]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#129 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#129 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#131] -Results [4]: [i_category#119, i_class#120, i_brand#121, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#129 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#131 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#75, isEmpty#76] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [4]: [i_category#18, i_class#17, i_brand#16, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (47) HashAggregate [codegen id : 52] -Input [4]: [i_category#119, i_class#120, i_brand#121, sumsales#41] -Keys [3]: [i_category#119, i_class#120, i_brand#121] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#132, isEmpty#133] -Results [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] +Input [4]: [i_category#18, i_class#17, i_brand#16, sumsales#30] +Keys [3]: [i_category#18, i_class#17, i_brand#16] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#77, isEmpty#78] +Results [5]: [i_category#18, i_class#17, i_brand#16, sum#79, isEmpty#80] (48) Exchange -Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] -Arguments: hashpartitioning(i_category#119, i_class#120, i_brand#121, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [5]: [i_category#18, i_class#17, i_brand#16, sum#79, isEmpty#80] +Arguments: hashpartitioning(i_category#18, i_class#17, i_brand#16, 5), ENSURE_REQUIREMENTS, [id=#81] (49) HashAggregate [codegen id : 53] -Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] -Keys [3]: [i_category#119, i_class#120, i_brand#121] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#137] -Results [9]: [i_category#119, i_class#120, i_brand#121, null AS i_product_name#138, null AS d_year#139, null AS d_qoy#140, null AS d_moy#141, null AS s_store_id#142, sum(sumsales#41)#137 AS sumsales#143] +Input [5]: [i_category#18, i_class#17, i_brand#16, sum#79, isEmpty#80] +Keys [3]: [i_category#18, i_class#17, i_brand#16] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#82] +Results [9]: [i_category#18, i_class#17, i_brand#16, null AS i_product_name#83, null AS d_year#84, null AS d_qoy#85, null AS d_moy#86, null AS s_store_id#87, sum(sumsales#30)#82 AS sumsales#88] (50) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#144, i_class#145, i_brand#146, i_product_name#147, d_year#148, d_qoy#149, d_moy#150, s_store_id#151, sum#152, isEmpty#153] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#89, isEmpty#90] (51) HashAggregate [codegen id : 61] -Input [10]: [i_category#144, i_class#145, i_brand#146, i_product_name#147, d_year#148, d_qoy#149, d_moy#150, s_store_id#151, sum#152, isEmpty#153] -Keys [8]: [i_category#144, i_class#145, i_brand#146, i_product_name#147, d_year#148, d_qoy#149, d_moy#150, s_store_id#151] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#154 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#155 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#154 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#155 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#156] -Results [3]: [i_category#144, i_class#145, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#154 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#155 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#156 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#89, isEmpty#90] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [3]: [i_category#18, i_class#17, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (52) HashAggregate [codegen id : 61] -Input [3]: [i_category#144, i_class#145, sumsales#41] -Keys [2]: [i_category#144, i_class#145] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#157, isEmpty#158] -Results [4]: [i_category#144, i_class#145, sum#159, isEmpty#160] +Input [3]: [i_category#18, i_class#17, sumsales#30] +Keys [2]: [i_category#18, i_class#17] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#91, isEmpty#92] +Results [4]: [i_category#18, i_class#17, sum#93, isEmpty#94] (53) Exchange -Input [4]: [i_category#144, i_class#145, sum#159, isEmpty#160] -Arguments: hashpartitioning(i_category#144, i_class#145, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [4]: [i_category#18, i_class#17, sum#93, isEmpty#94] +Arguments: hashpartitioning(i_category#18, i_class#17, 5), ENSURE_REQUIREMENTS, [id=#95] (54) HashAggregate [codegen id : 62] -Input [4]: [i_category#144, i_class#145, sum#159, isEmpty#160] -Keys [2]: [i_category#144, i_class#145] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#162] -Results [9]: [i_category#144, i_class#145, null AS i_brand#163, null AS i_product_name#164, null AS d_year#165, null AS d_qoy#166, null AS d_moy#167, null AS s_store_id#168, sum(sumsales#41)#162 AS sumsales#169] +Input [4]: [i_category#18, i_class#17, sum#93, isEmpty#94] +Keys [2]: [i_category#18, i_class#17] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#96] +Results [9]: [i_category#18, i_class#17, null AS i_brand#97, null AS i_product_name#98, null AS d_year#99, null AS d_qoy#100, null AS d_moy#101, null AS s_store_id#102, sum(sumsales#30)#96 AS sumsales#103] (55) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#170, i_class#171, i_brand#172, i_product_name#173, d_year#174, d_qoy#175, d_moy#176, s_store_id#177, sum#178, isEmpty#179] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#104, isEmpty#105] (56) HashAggregate [codegen id : 70] -Input [10]: [i_category#170, i_class#171, i_brand#172, i_product_name#173, d_year#174, d_qoy#175, d_moy#176, s_store_id#177, sum#178, isEmpty#179] -Keys [8]: [i_category#170, i_class#171, i_brand#172, i_product_name#173, d_year#174, d_qoy#175, d_moy#176, s_store_id#177] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#180 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#181 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#180 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#181 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#182] -Results [2]: [i_category#170, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#180 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#181 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#182 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#104, isEmpty#105] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [2]: [i_category#18, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (57) HashAggregate [codegen id : 70] -Input [2]: [i_category#170, sumsales#41] -Keys [1]: [i_category#170] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#183, isEmpty#184] -Results [3]: [i_category#170, sum#185, isEmpty#186] +Input [2]: [i_category#18, sumsales#30] +Keys [1]: [i_category#18] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#106, isEmpty#107] +Results [3]: [i_category#18, sum#108, isEmpty#109] (58) Exchange -Input [3]: [i_category#170, sum#185, isEmpty#186] -Arguments: hashpartitioning(i_category#170, 5), ENSURE_REQUIREMENTS, [id=#187] +Input [3]: [i_category#18, sum#108, isEmpty#109] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#110] (59) HashAggregate [codegen id : 71] -Input [3]: [i_category#170, sum#185, isEmpty#186] -Keys [1]: [i_category#170] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#188] -Results [9]: [i_category#170, null AS i_class#189, null AS i_brand#190, null AS i_product_name#191, null AS d_year#192, null AS d_qoy#193, null AS d_moy#194, null AS s_store_id#195, sum(sumsales#41)#188 AS sumsales#196] +Input [3]: [i_category#18, sum#108, isEmpty#109] +Keys [1]: [i_category#18] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#111] +Results [9]: [i_category#18, null AS i_class#112, null AS i_brand#113, null AS i_product_name#114, null AS d_year#115, null AS d_qoy#116, null AS d_moy#117, null AS s_store_id#118, sum(sumsales#30)#111 AS sumsales#119] (60) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#197, i_class#198, i_brand#199, i_product_name#200, d_year#201, d_qoy#202, d_moy#203, s_store_id#204, sum#205, isEmpty#206] +Output [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#120, isEmpty#121] (61) HashAggregate [codegen id : 79] -Input [10]: [i_category#197, i_class#198, i_brand#199, i_product_name#200, d_year#201, d_qoy#202, d_moy#203, s_store_id#204, sum#205, isEmpty#206] -Keys [8]: [i_category#197, i_class#198, i_brand#199, i_product_name#200, d_year#201, d_qoy#202, d_moy#203, s_store_id#204] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#207 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#208 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#207 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#208 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#209] -Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#207 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#208 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#209 AS sumsales#41] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#120, isEmpty#121] +Keys [8]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#26 AS sumsales#30] (62) HashAggregate [codegen id : 79] -Input [1]: [sumsales#41] +Input [1]: [sumsales#30] Keys: [] -Functions [1]: [partial_sum(sumsales#41)] -Aggregate Attributes [2]: [sum#210, isEmpty#211] -Results [2]: [sum#212, isEmpty#213] +Functions [1]: [partial_sum(sumsales#30)] +Aggregate Attributes [2]: [sum#122, isEmpty#123] +Results [2]: [sum#124, isEmpty#125] (63) Exchange -Input [2]: [sum#212, isEmpty#213] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#214] +Input [2]: [sum#124, isEmpty#125] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] (64) HashAggregate [codegen id : 80] -Input [2]: [sum#212, isEmpty#213] +Input [2]: [sum#124, isEmpty#125] Keys: [] -Functions [1]: [sum(sumsales#41)] -Aggregate Attributes [1]: [sum(sumsales#41)#215] -Results [9]: [null AS i_category#216, null AS i_class#217, null AS i_brand#218, null AS i_product_name#219, null AS d_year#220, null AS d_qoy#221, null AS d_moy#222, null AS s_store_id#223, sum(sumsales#41)#215 AS sumsales#224] +Functions [1]: [sum(sumsales#30)] +Aggregate Attributes [1]: [sum(sumsales#30)#127] +Results [9]: [null AS i_category#128, null AS i_class#129, null AS i_brand#130, null AS i_product_name#131, null AS d_year#132, null AS d_qoy#133, null AS d_moy#134, null AS s_store_id#135, sum(sumsales#30)#127 AS sumsales#136] (65) Union (66) Exchange Input [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#225] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#137] (67) Sort [codegen id : 81] Input [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27] @@ -418,15 +418,15 @@ Arguments: [i_category#18 ASC NULLS FIRST, sumsales#27 DESC NULLS LAST], false, (68) Window Input [9]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27] -Arguments: [rank(sumsales#27) windowspecdefinition(i_category#18, sumsales#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#226], [i_category#18], [sumsales#27 DESC NULLS LAST] +Arguments: [rank(sumsales#27) windowspecdefinition(i_category#18, sumsales#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#138], [i_category#18], [sumsales#27 DESC NULLS LAST] (69) Filter [codegen id : 82] -Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#226] -Condition : (rk#226 <= 100) +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138] +Condition : (rk#138 <= 100) (70) TakeOrderedAndProject -Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#226] -Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_product_name#19 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#27 ASC NULLS FIRST, rk#226 ASC NULLS FIRST], [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#226] +Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138] +Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_product_name#19 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#27 ASC NULLS FIRST, rk#138 ASC NULLS FIRST], [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138] ===== Subqueries ===== @@ -439,25 +439,25 @@ BroadcastExchange (75) (71) Scan parquet default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#227, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#139, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#227, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#139, d_year#8, d_moy#9, d_qoy#10] (73) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#227, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#227) AND (d_month_seq#227 >= 1212)) AND (d_month_seq#227 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#139, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#139) AND (d_month_seq#139 >= 1212)) AND (d_month_seq#139 <= 1223)) AND isnotnull(d_date_sk#7)) (74) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#227, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#139, d_year#8, d_moy#9, d_qoy#10] (75) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#228] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#140] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index bc4bb64349ac6..48ab2f77ad964 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -168,234 +168,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 as decimal(38,2)) AS sumsales#26] (22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, s_store_id#34, sum#35, isEmpty#36] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#27, isEmpty#28] (23) HashAggregate [codegen id : 10] -Input [10]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, s_store_id#34, sum#35, isEmpty#36] -Keys [8]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, s_store_id#34] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#37 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#37 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#39] -Results [8]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#37 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#39 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#27, isEmpty#28] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (24) HashAggregate [codegen id : 10] -Input [8]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, sumsales#40] -Keys [7]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [9]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, sum#43, isEmpty#44] +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, sumsales#29] +Keys [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#30, isEmpty#31] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, sum#32, isEmpty#33] (25) Exchange -Input [9]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, sum#32, isEmpty#33] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, [id=#34] (26) HashAggregate [codegen id : 11] -Input [9]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, sum#43, isEmpty#44] -Keys [7]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#46] -Results [9]: [i_category#27, i_class#28, i_brand#29, i_product_name#30, d_year#31, d_qoy#32, d_moy#33, null AS s_store_id#47, sum(sumsales#40)#46 AS sumsales#48] +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, sum#32, isEmpty#33] +Keys [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#35] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, null AS s_store_id#36, sum(sumsales#29)#35 AS sumsales#37] (27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, d_moy#55, s_store_id#56, sum#57, isEmpty#58] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#38, isEmpty#39] (28) HashAggregate [codegen id : 16] -Input [10]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, d_moy#55, s_store_id#56, sum#57, isEmpty#58] -Keys [8]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, d_moy#55, s_store_id#56] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#59 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#59 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#61] -Results [7]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#59 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#60 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#61 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#38, isEmpty#39] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (29) HashAggregate [codegen id : 16] -Input [7]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, sumsales#40] -Keys [6]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#62, isEmpty#63] -Results [8]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, sum#64, isEmpty#65] +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, sumsales#29] +Keys [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, sum#42, isEmpty#43] (30) Exchange -Input [8]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, sum#64, isEmpty#65] -Arguments: hashpartitioning(i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, sum#42, isEmpty#43] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, [id=#44] (31) HashAggregate [codegen id : 17] -Input [8]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, sum#64, isEmpty#65] -Keys [6]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#67] -Results [9]: [i_category#49, i_class#50, i_brand#51, i_product_name#52, d_year#53, d_qoy#54, null AS d_moy#68, null AS s_store_id#69, sum(sumsales#40)#67 AS sumsales#70] +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, sum#42, isEmpty#43] +Keys [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#45] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, null AS d_moy#46, null AS s_store_id#47, sum(sumsales#29)#45 AS sumsales#48] (32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78, sum#79, isEmpty#80] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#49, isEmpty#50] (33) HashAggregate [codegen id : 22] -Input [10]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78, sum#79, isEmpty#80] -Keys [8]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#81 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#82 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#81 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#82 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#83] -Results [6]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#81 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#82 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#83 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#49, isEmpty#50] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (34) HashAggregate [codegen id : 22] -Input [6]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, sumsales#40] -Keys [5]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#84, isEmpty#85] -Results [7]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, sum#86, isEmpty#87] +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, sumsales#29] +Keys [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, sum#53, isEmpty#54] (35) Exchange -Input [7]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, sum#86, isEmpty#87] -Arguments: hashpartitioning(i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, sum#53, isEmpty#54] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, 5), ENSURE_REQUIREMENTS, [id=#55] (36) HashAggregate [codegen id : 23] -Input [7]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, sum#86, isEmpty#87] -Keys [5]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#89] -Results [9]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, null AS d_qoy#90, null AS d_moy#91, null AS s_store_id#92, sum(sumsales#40)#89 AS sumsales#93] +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, sum#53, isEmpty#54] +Keys [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#56] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, null AS d_qoy#57, null AS d_moy#58, null AS s_store_id#59, sum(sumsales#29)#56 AS sumsales#60] (37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, d_year#98, d_qoy#99, d_moy#100, s_store_id#101, sum#102, isEmpty#103] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#61, isEmpty#62] (38) HashAggregate [codegen id : 28] -Input [10]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, d_year#98, d_qoy#99, d_moy#100, s_store_id#101, sum#102, isEmpty#103] -Keys [8]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, d_year#98, d_qoy#99, d_moy#100, s_store_id#101] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#104 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#105 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#104 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#105 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#106] -Results [5]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#104 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#105 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#106 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#61, isEmpty#62] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (39) HashAggregate [codegen id : 28] -Input [5]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, sumsales#40] -Keys [4]: [i_category#94, i_class#95, i_brand#96, i_product_name#97] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#107, isEmpty#108] -Results [6]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, sum#109, isEmpty#110] +Input [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sumsales#29] +Keys [4]: [i_category#17, i_class#16, i_brand#15, i_product_name#18] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#65, isEmpty#66] (40) Exchange -Input [6]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, sum#109, isEmpty#110] -Arguments: hashpartitioning(i_category#94, i_class#95, i_brand#96, i_product_name#97, 5), ENSURE_REQUIREMENTS, [id=#111] +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#65, isEmpty#66] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, 5), ENSURE_REQUIREMENTS, [id=#67] (41) HashAggregate [codegen id : 29] -Input [6]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, sum#109, isEmpty#110] -Keys [4]: [i_category#94, i_class#95, i_brand#96, i_product_name#97] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#112] -Results [9]: [i_category#94, i_class#95, i_brand#96, i_product_name#97, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#40)#112 AS sumsales#117] +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#65, isEmpty#66] +Keys [4]: [i_category#17, i_class#16, i_brand#15, i_product_name#18] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#68] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, null AS d_year#69, null AS d_qoy#70, null AS d_moy#71, null AS s_store_id#72, sum(sumsales#29)#68 AS sumsales#73] (42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#74, isEmpty#75] (43) HashAggregate [codegen id : 34] -Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] -Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#128 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#129 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#128 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#129 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#130] -Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#128 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#129 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#130 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#74, isEmpty#75] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [4]: [i_category#17, i_class#16, i_brand#15, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (44) HashAggregate [codegen id : 34] -Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#40] -Keys [3]: [i_category#118, i_class#119, i_brand#120] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Input [4]: [i_category#17, i_class#16, i_brand#15, sumsales#29] +Keys [3]: [i_category#17, i_class#16, i_brand#15] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [5]: [i_category#17, i_class#16, i_brand#15, sum#78, isEmpty#79] (45) Exchange -Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] -Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [5]: [i_category#17, i_class#16, i_brand#15, sum#78, isEmpty#79] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, 5), ENSURE_REQUIREMENTS, [id=#80] (46) HashAggregate [codegen id : 35] -Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] -Keys [3]: [i_category#118, i_class#119, i_brand#120] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#136] -Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#137, null AS d_year#138, null AS d_qoy#139, null AS d_moy#140, null AS s_store_id#141, sum(sumsales#40)#136 AS sumsales#142] +Input [5]: [i_category#17, i_class#16, i_brand#15, sum#78, isEmpty#79] +Keys [3]: [i_category#17, i_class#16, i_brand#15] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#81] +Results [9]: [i_category#17, i_class#16, i_brand#15, null AS i_product_name#82, null AS d_year#83, null AS d_qoy#84, null AS d_moy#85, null AS s_store_id#86, sum(sumsales#29)#81 AS sumsales#87] (47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sum#151, isEmpty#152] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#88, isEmpty#89] (48) HashAggregate [codegen id : 40] -Input [10]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sum#151, isEmpty#152] -Keys [8]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#153 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#154 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#153 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#154 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#155] -Results [3]: [i_category#143, i_class#144, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#153 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#154 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#155 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#88, isEmpty#89] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [3]: [i_category#17, i_class#16, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (49) HashAggregate [codegen id : 40] -Input [3]: [i_category#143, i_class#144, sumsales#40] -Keys [2]: [i_category#143, i_class#144] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#156, isEmpty#157] -Results [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] +Input [3]: [i_category#17, i_class#16, sumsales#29] +Keys [2]: [i_category#17, i_class#16] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#90, isEmpty#91] +Results [4]: [i_category#17, i_class#16, sum#92, isEmpty#93] (50) Exchange -Input [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] -Arguments: hashpartitioning(i_category#143, i_class#144, 5), ENSURE_REQUIREMENTS, [id=#160] +Input [4]: [i_category#17, i_class#16, sum#92, isEmpty#93] +Arguments: hashpartitioning(i_category#17, i_class#16, 5), ENSURE_REQUIREMENTS, [id=#94] (51) HashAggregate [codegen id : 41] -Input [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] -Keys [2]: [i_category#143, i_class#144] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#161] -Results [9]: [i_category#143, i_class#144, null AS i_brand#162, null AS i_product_name#163, null AS d_year#164, null AS d_qoy#165, null AS d_moy#166, null AS s_store_id#167, sum(sumsales#40)#161 AS sumsales#168] +Input [4]: [i_category#17, i_class#16, sum#92, isEmpty#93] +Keys [2]: [i_category#17, i_class#16] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#95] +Results [9]: [i_category#17, i_class#16, null AS i_brand#96, null AS i_product_name#97, null AS d_year#98, null AS d_qoy#99, null AS d_moy#100, null AS s_store_id#101, sum(sumsales#29)#95 AS sumsales#102] (52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#169, i_class#170, i_brand#171, i_product_name#172, d_year#173, d_qoy#174, d_moy#175, s_store_id#176, sum#177, isEmpty#178] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#103, isEmpty#104] (53) HashAggregate [codegen id : 46] -Input [10]: [i_category#169, i_class#170, i_brand#171, i_product_name#172, d_year#173, d_qoy#174, d_moy#175, s_store_id#176, sum#177, isEmpty#178] -Keys [8]: [i_category#169, i_class#170, i_brand#171, i_product_name#172, d_year#173, d_qoy#174, d_moy#175, s_store_id#176] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#179 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#180 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#179 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#180 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#181] -Results [2]: [i_category#169, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#179 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#180 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#181 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#103, isEmpty#104] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [2]: [i_category#17, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (54) HashAggregate [codegen id : 46] -Input [2]: [i_category#169, sumsales#40] -Keys [1]: [i_category#169] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#182, isEmpty#183] -Results [3]: [i_category#169, sum#184, isEmpty#185] +Input [2]: [i_category#17, sumsales#29] +Keys [1]: [i_category#17] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#105, isEmpty#106] +Results [3]: [i_category#17, sum#107, isEmpty#108] (55) Exchange -Input [3]: [i_category#169, sum#184, isEmpty#185] -Arguments: hashpartitioning(i_category#169, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [3]: [i_category#17, sum#107, isEmpty#108] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [id=#109] (56) HashAggregate [codegen id : 47] -Input [3]: [i_category#169, sum#184, isEmpty#185] -Keys [1]: [i_category#169] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#187] -Results [9]: [i_category#169, null AS i_class#188, null AS i_brand#189, null AS i_product_name#190, null AS d_year#191, null AS d_qoy#192, null AS d_moy#193, null AS s_store_id#194, sum(sumsales#40)#187 AS sumsales#195] +Input [3]: [i_category#17, sum#107, isEmpty#108] +Keys [1]: [i_category#17] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#110] +Results [9]: [i_category#17, null AS i_class#111, null AS i_brand#112, null AS i_product_name#113, null AS d_year#114, null AS d_qoy#115, null AS d_moy#116, null AS s_store_id#117, sum(sumsales#29)#110 AS sumsales#118] (57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#196, i_class#197, i_brand#198, i_product_name#199, d_year#200, d_qoy#201, d_moy#202, s_store_id#203, sum#204, isEmpty#205] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#119, isEmpty#120] (58) HashAggregate [codegen id : 52] -Input [10]: [i_category#196, i_class#197, i_brand#198, i_product_name#199, d_year#200, d_qoy#201, d_moy#202, s_store_id#203, sum#204, isEmpty#205] -Keys [8]: [i_category#196, i_class#197, i_brand#198, i_product_name#199, d_year#200, d_qoy#201, d_moy#202, s_store_id#203] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#206 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#207 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] -Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#206 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#207 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#208] -Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#206 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#207 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#208 AS sumsales#40] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#119, isEmpty#120] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))] +Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#3 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#25 AS sumsales#29] (59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#40] +Input [1]: [sumsales#29] Keys: [] -Functions [1]: [partial_sum(sumsales#40)] -Aggregate Attributes [2]: [sum#209, isEmpty#210] -Results [2]: [sum#211, isEmpty#212] +Functions [1]: [partial_sum(sumsales#29)] +Aggregate Attributes [2]: [sum#121, isEmpty#122] +Results [2]: [sum#123, isEmpty#124] (60) Exchange -Input [2]: [sum#211, isEmpty#212] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#213] +Input [2]: [sum#123, isEmpty#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (61) HashAggregate [codegen id : 53] -Input [2]: [sum#211, isEmpty#212] +Input [2]: [sum#123, isEmpty#124] Keys: [] -Functions [1]: [sum(sumsales#40)] -Aggregate Attributes [1]: [sum(sumsales#40)#214] -Results [9]: [null AS i_category#215, null AS i_class#216, null AS i_brand#217, null AS i_product_name#218, null AS d_year#219, null AS d_qoy#220, null AS d_moy#221, null AS s_store_id#222, sum(sumsales#40)#214 AS sumsales#223] +Functions [1]: [sum(sumsales#29)] +Aggregate Attributes [1]: [sum(sumsales#29)#126] +Results [9]: [null AS i_category#127, null AS i_class#128, null AS i_brand#129, null AS i_product_name#130, null AS d_year#131, null AS d_qoy#132, null AS d_moy#133, null AS s_store_id#134, sum(sumsales#29)#126 AS sumsales#135] (62) Union (63) Exchange Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [id=#224] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [id=#136] (64) Sort [codegen id : 54] Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26] @@ -403,15 +403,15 @@ Arguments: [i_category#17 ASC NULLS FIRST, sumsales#26 DESC NULLS LAST], false, (65) Window Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26] -Arguments: [rank(sumsales#26) windowspecdefinition(i_category#17, sumsales#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#225], [i_category#17], [sumsales#26 DESC NULLS LAST] +Arguments: [rank(sumsales#26) windowspecdefinition(i_category#17, sumsales#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#137], [i_category#17], [sumsales#26 DESC NULLS LAST] (66) Filter [codegen id : 55] -Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#225] -Condition : (rk#225 <= 100) +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137] +Condition : (rk#137 <= 100) (67) TakeOrderedAndProject -Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#225] -Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#26 ASC NULLS FIRST, rk#225 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#225] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137] +Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#26 ASC NULLS FIRST, rk#137 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137] ===== Subqueries ===== @@ -424,25 +424,25 @@ BroadcastExchange (72) (68) Scan parquet default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#138, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#138, d_year#8, d_moy#9, d_qoy#10] (70) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#226) AND (d_month_seq#226 >= 1212)) AND (d_month_seq#226 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#138, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#138) AND (d_month_seq#138 >= 1212)) AND (d_month_seq#138 <= 1223)) AND isnotnull(d_date_sk#7)) (71) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#138, d_year#8, d_moy#9, d_qoy#10] (72) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#227] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 883130da8087f..bd893d09165e1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -226,60 +226,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) as decimal(27,2)) AS total_sum#28, s_state#8, s_county#7, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] (37) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#32, s_county#33, sum#34] +Output [3]: [s_state#8, s_county#7, sum#32] (38) HashAggregate [codegen id : 18] -Input [3]: [s_state#32, s_county#33, sum#34] -Keys [2]: [s_state#32, s_county#33] -Functions [1]: [sum(UnscaledValue(ss_net_profit#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#35))#36] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#35))#36,17,2) AS total_sum#37, s_state#32] +Input [3]: [s_state#8, s_county#7, sum#32] +Keys [2]: [s_state#8, s_county#7] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#33, s_state#8] (39) HashAggregate [codegen id : 18] -Input [2]: [total_sum#37, s_state#32] -Keys [1]: [s_state#32] -Functions [1]: [partial_sum(total_sum#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [3]: [s_state#32, sum#40, isEmpty#41] +Input [2]: [total_sum#33, s_state#8] +Keys [1]: [s_state#8] +Functions [1]: [partial_sum(total_sum#33)] +Aggregate Attributes [2]: [sum#34, isEmpty#35] +Results [3]: [s_state#8, sum#36, isEmpty#37] (40) Exchange -Input [3]: [s_state#32, sum#40, isEmpty#41] -Arguments: hashpartitioning(s_state#32, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [s_state#8, sum#36, isEmpty#37] +Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [id=#38] (41) HashAggregate [codegen id : 19] -Input [3]: [s_state#32, sum#40, isEmpty#41] -Keys [1]: [s_state#32] -Functions [1]: [sum(total_sum#37)] -Aggregate Attributes [1]: [sum(total_sum#37)#43] -Results [6]: [sum(total_sum#37)#43 AS total_sum#44, s_state#32, null AS s_county#45, 0 AS g_state#46, 1 AS g_county#47, 1 AS lochierarchy#48] +Input [3]: [s_state#8, sum#36, isEmpty#37] +Keys [1]: [s_state#8] +Functions [1]: [sum(total_sum#33)] +Aggregate Attributes [1]: [sum(total_sum#33)#39] +Results [6]: [sum(total_sum#33)#39 AS total_sum#40, s_state#8, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] (42) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#49, s_county#50, sum#51] +Output [3]: [s_state#8, s_county#7, sum#45] (43) HashAggregate [codegen id : 28] -Input [3]: [s_state#49, s_county#50, sum#51] -Keys [2]: [s_state#49, s_county#50] -Functions [1]: [sum(UnscaledValue(ss_net_profit#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#52))#53] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#53,17,2) AS total_sum#37] +Input [3]: [s_state#8, s_county#7, sum#45] +Keys [2]: [s_state#8, s_county#7] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#33] (44) HashAggregate [codegen id : 28] -Input [1]: [total_sum#37] +Input [1]: [total_sum#33] Keys: [] -Functions [1]: [partial_sum(total_sum#37)] -Aggregate Attributes [2]: [sum#54, isEmpty#55] -Results [2]: [sum#56, isEmpty#57] +Functions [1]: [partial_sum(total_sum#33)] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [2]: [sum#48, isEmpty#49] (45) Exchange -Input [2]: [sum#56, isEmpty#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sum#48, isEmpty#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] (46) HashAggregate [codegen id : 29] -Input [2]: [sum#56, isEmpty#57] +Input [2]: [sum#48, isEmpty#49] Keys: [] -Functions [1]: [sum(total_sum#37)] -Aggregate Attributes [1]: [sum(total_sum#37)#59] -Results [6]: [sum(total_sum#37)#59 AS total_sum#60, null AS s_state#61, null AS s_county#62, 1 AS g_state#63, 1 AS g_county#64, 2 AS lochierarchy#65] +Functions [1]: [sum(total_sum#33)] +Aggregate Attributes [1]: [sum(total_sum#33)#51] +Results [6]: [sum(total_sum#33)#51 AS total_sum#52, null AS s_state#53, null AS s_county#54, 1 AS g_state#55, 1 AS g_county#56, 2 AS lochierarchy#57] (47) Union @@ -292,34 +292,34 @@ Results [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, loch (49) Exchange Input [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31] -Arguments: hashpartitioning(total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, [id=#66] +Arguments: hashpartitioning(total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, [id=#58] (50) HashAggregate [codegen id : 31] Input [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31] Keys [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, CASE WHEN (g_county#30 = 0) THEN s_state#8 END AS _w0#67] +Results [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, CASE WHEN (g_county#30 = 0) THEN s_state#8 END AS _w0#59] (51) Exchange -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67] -Arguments: hashpartitioning(lochierarchy#31, _w0#67, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59] +Arguments: hashpartitioning(lochierarchy#31, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 32] -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67] -Arguments: [lochierarchy#31 ASC NULLS FIRST, _w0#67 ASC NULLS FIRST, total_sum#28 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59] +Arguments: [lochierarchy#31 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#28 DESC NULLS LAST], false, 0 (53) Window -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67] -Arguments: [rank(total_sum#28) windowspecdefinition(lochierarchy#31, _w0#67, total_sum#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#69], [lochierarchy#31, _w0#67], [total_sum#28 DESC NULLS LAST] +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59] +Arguments: [rank(total_sum#28) windowspecdefinition(lochierarchy#31, _w0#59, total_sum#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#31, _w0#59], [total_sum#28 DESC NULLS LAST] (54) Project [codegen id : 33] -Output [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#69] -Input [6]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67, rank_within_parent#69] +Output [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#61] +Input [6]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59, rank_within_parent#61] (55) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#69] -Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#69 ASC NULLS FIRST], [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#69] +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#61] +Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#61] ===== Subqueries ===== @@ -332,26 +332,26 @@ BroadcastExchange (60) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#70] +Output [2]: [d_date_sk#5, d_month_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#70] +Input [2]: [d_date_sk#5, d_month_seq#62] (58) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#70] -Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1212)) AND (d_month_seq#70 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#62] +Condition : (((isnotnull(d_month_seq#62) AND (d_month_seq#62 >= 1212)) AND (d_month_seq#62 <= 1223)) AND isnotnull(d_date_sk#5)) (59) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_month_seq#70] +Input [2]: [d_date_sk#5, d_month_seq#62] (60) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index c5cb2010c67dd..2bfa05e382559 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -226,60 +226,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) as decimal(27,2)) AS total_sum#28, s_state#8, s_county#7, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] (37) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#32, s_county#33, sum#34] +Output [3]: [s_state#8, s_county#7, sum#32] (38) HashAggregate [codegen id : 18] -Input [3]: [s_state#32, s_county#33, sum#34] -Keys [2]: [s_state#32, s_county#33] -Functions [1]: [sum(UnscaledValue(ss_net_profit#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#35))#36] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#35))#36,17,2) AS total_sum#37, s_state#32] +Input [3]: [s_state#8, s_county#7, sum#32] +Keys [2]: [s_state#8, s_county#7] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#33, s_state#8] (39) HashAggregate [codegen id : 18] -Input [2]: [total_sum#37, s_state#32] -Keys [1]: [s_state#32] -Functions [1]: [partial_sum(total_sum#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [3]: [s_state#32, sum#40, isEmpty#41] +Input [2]: [total_sum#33, s_state#8] +Keys [1]: [s_state#8] +Functions [1]: [partial_sum(total_sum#33)] +Aggregate Attributes [2]: [sum#34, isEmpty#35] +Results [3]: [s_state#8, sum#36, isEmpty#37] (40) Exchange -Input [3]: [s_state#32, sum#40, isEmpty#41] -Arguments: hashpartitioning(s_state#32, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [s_state#8, sum#36, isEmpty#37] +Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [id=#38] (41) HashAggregate [codegen id : 19] -Input [3]: [s_state#32, sum#40, isEmpty#41] -Keys [1]: [s_state#32] -Functions [1]: [sum(total_sum#37)] -Aggregate Attributes [1]: [sum(total_sum#37)#43] -Results [6]: [sum(total_sum#37)#43 AS total_sum#44, s_state#32, null AS s_county#45, 0 AS g_state#46, 1 AS g_county#47, 1 AS lochierarchy#48] +Input [3]: [s_state#8, sum#36, isEmpty#37] +Keys [1]: [s_state#8] +Functions [1]: [sum(total_sum#33)] +Aggregate Attributes [1]: [sum(total_sum#33)#39] +Results [6]: [sum(total_sum#33)#39 AS total_sum#40, s_state#8, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] (42) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#49, s_county#50, sum#51] +Output [3]: [s_state#8, s_county#7, sum#45] (43) HashAggregate [codegen id : 28] -Input [3]: [s_state#49, s_county#50, sum#51] -Keys [2]: [s_state#49, s_county#50] -Functions [1]: [sum(UnscaledValue(ss_net_profit#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#52))#53] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#53,17,2) AS total_sum#37] +Input [3]: [s_state#8, s_county#7, sum#45] +Keys [2]: [s_state#8, s_county#7] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#33] (44) HashAggregate [codegen id : 28] -Input [1]: [total_sum#37] +Input [1]: [total_sum#33] Keys: [] -Functions [1]: [partial_sum(total_sum#37)] -Aggregate Attributes [2]: [sum#54, isEmpty#55] -Results [2]: [sum#56, isEmpty#57] +Functions [1]: [partial_sum(total_sum#33)] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [2]: [sum#48, isEmpty#49] (45) Exchange -Input [2]: [sum#56, isEmpty#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sum#48, isEmpty#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] (46) HashAggregate [codegen id : 29] -Input [2]: [sum#56, isEmpty#57] +Input [2]: [sum#48, isEmpty#49] Keys: [] -Functions [1]: [sum(total_sum#37)] -Aggregate Attributes [1]: [sum(total_sum#37)#59] -Results [6]: [sum(total_sum#37)#59 AS total_sum#60, null AS s_state#61, null AS s_county#62, 1 AS g_state#63, 1 AS g_county#64, 2 AS lochierarchy#65] +Functions [1]: [sum(total_sum#33)] +Aggregate Attributes [1]: [sum(total_sum#33)#51] +Results [6]: [sum(total_sum#33)#51 AS total_sum#52, null AS s_state#53, null AS s_county#54, 1 AS g_state#55, 1 AS g_county#56, 2 AS lochierarchy#57] (47) Union @@ -292,34 +292,34 @@ Results [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, loch (49) Exchange Input [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31] -Arguments: hashpartitioning(total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, [id=#66] +Arguments: hashpartitioning(total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, [id=#58] (50) HashAggregate [codegen id : 31] Input [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31] Keys [6]: [total_sum#28, s_state#8, s_county#7, g_state#29, g_county#30, lochierarchy#31] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, CASE WHEN (g_county#30 = 0) THEN s_state#8 END AS _w0#67] +Results [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, CASE WHEN (g_county#30 = 0) THEN s_state#8 END AS _w0#59] (51) Exchange -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67] -Arguments: hashpartitioning(lochierarchy#31, _w0#67, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59] +Arguments: hashpartitioning(lochierarchy#31, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 32] -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67] -Arguments: [lochierarchy#31 ASC NULLS FIRST, _w0#67 ASC NULLS FIRST, total_sum#28 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59] +Arguments: [lochierarchy#31 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#28 DESC NULLS LAST], false, 0 (53) Window -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67] -Arguments: [rank(total_sum#28) windowspecdefinition(lochierarchy#31, _w0#67, total_sum#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#69], [lochierarchy#31, _w0#67], [total_sum#28 DESC NULLS LAST] +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59] +Arguments: [rank(total_sum#28) windowspecdefinition(lochierarchy#31, _w0#59, total_sum#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#31, _w0#59], [total_sum#28 DESC NULLS LAST] (54) Project [codegen id : 33] -Output [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#69] -Input [6]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#67, rank_within_parent#69] +Output [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#61] +Input [6]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, _w0#59, rank_within_parent#61] (55) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#69] -Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#69 ASC NULLS FIRST], [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#69] +Input [5]: [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#61] +Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#28, s_state#8, s_county#7, lochierarchy#31, rank_within_parent#61] ===== Subqueries ===== @@ -332,26 +332,26 @@ BroadcastExchange (60) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#70] +Output [2]: [d_date_sk#5, d_month_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#70] +Input [2]: [d_date_sk#5, d_month_seq#62] (58) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#70] -Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1212)) AND (d_month_seq#70 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#62] +Condition : (((isnotnull(d_month_seq#62) AND (d_month_seq#62 >= 1212)) AND (d_month_seq#62 <= 1223)) AND isnotnull(d_date_sk#5)) (59) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_month_seq#70] +Input [2]: [d_date_sk#5, d_month_seq#62] (60) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index 0ba4816e69f99..eda73d02cec65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -242,201 +242,201 @@ Arguments: hashpartitioning(c_customer_id#28, c_first_name#29, c_last_name#30, d Input [5]: [c_customer_id#28, c_first_name#29, c_last_name#30, d_year#25, sum#32] Keys [4]: [c_customer_id#28, c_first_name#29, c_last_name#30, d_year#25] Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#34] -Results [4]: [c_customer_id#28 AS customer_id#35, c_first_name#29 AS customer_first_name#36, c_last_name#30 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#34,17,2) AS year_total#38] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#16] +Results [4]: [c_customer_id#28 AS customer_id#34, c_first_name#29 AS customer_first_name#35, c_last_name#30 AS customer_last_name#36, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#16,17,2) AS year_total#37] (37) Exchange -Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] -Arguments: hashpartitioning(customer_id#35, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37] +Arguments: hashpartitioning(customer_id#34, 5), ENSURE_REQUIREMENTS, [id=#38] (38) Sort [codegen id : 16] -Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] -Arguments: [customer_id#35 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37] +Arguments: [customer_id#34 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#35] +Right keys [1]: [customer_id#34] Join condition: None (40) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#40, ws_net_paid#41, ws_sold_date_sk#42] +Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#40, ws_net_paid#41, ws_sold_date_sk#42] +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] (42) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#40, ws_net_paid#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#40) +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_bill_customer_sk#39) (43) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#42, d_year#43] (44) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#42] Join condition: None (45) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#40, ws_net_paid#41, d_year#44] -Input [5]: [ws_bill_customer_sk#40, ws_net_paid#41, ws_sold_date_sk#42, d_date_sk#43, d_year#44] +Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, d_year#43] +Input [5]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#42, d_year#43] (46) Exchange -Input [3]: [ws_bill_customer_sk#40, ws_net_paid#41, d_year#44] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, d_year#43] +Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (47) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#40, ws_net_paid#41, d_year#44] -Arguments: [ws_bill_customer_sk#40 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, d_year#43] +Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 (48) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] +Output [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] (49) Sort [codegen id : 22] -Input [4]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] -Arguments: [c_customer_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] +Arguments: [c_customer_sk#45 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#40] -Right keys [1]: [c_customer_sk#46] +Left keys [1]: [ws_bill_customer_sk#39] +Right keys [1]: [c_customer_sk#45] Join condition: None (51) Project [codegen id : 23] -Output [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#41, d_year#44] -Input [7]: [ws_bill_customer_sk#40, ws_net_paid#41, d_year#44, c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49] +Output [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#40, d_year#43] +Input [7]: [ws_bill_customer_sk#39, ws_net_paid#40, d_year#43, c_customer_sk#45, c_customer_id#46, c_first_name#47, c_last_name#48] (52) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, ws_net_paid#41, d_year#44] -Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#41))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#44, sum#51] +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, ws_net_paid#40, d_year#43] +Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#43] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum#49] +Results [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#43, sum#50] (53) Exchange -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#44, sum#51] -Arguments: hashpartitioning(c_customer_id#47, c_first_name#48, c_last_name#49, d_year#44, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#43, sum#50] +Arguments: hashpartitioning(c_customer_id#46, c_first_name#47, c_last_name#48, d_year#43, 5), ENSURE_REQUIREMENTS, [id=#51] (54) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#44, sum#51] -Keys [4]: [c_customer_id#47, c_first_name#48, c_last_name#49, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#41))#53] -Results [2]: [c_customer_id#47 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#41))#53,17,2) AS year_total#55] +Input [5]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#43, sum#50] +Keys [4]: [c_customer_id#46, c_first_name#47, c_last_name#48, d_year#43] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#52] +Results [2]: [c_customer_id#46 AS customer_id#53, MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#52,17,2) AS year_total#54] (55) Filter [codegen id : 24] -Input [2]: [customer_id#54, year_total#55] -Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) +Input [2]: [customer_id#53, year_total#54] +Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) (56) Exchange -Input [2]: [customer_id#54, year_total#55] -Arguments: hashpartitioning(customer_id#54, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [2]: [customer_id#53, year_total#54] +Arguments: hashpartitioning(customer_id#53, 5), ENSURE_REQUIREMENTS, [id=#55] (57) Sort [codegen id : 25] -Input [2]: [customer_id#54, year_total#55] -Arguments: [customer_id#54 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#53, year_total#54] +Arguments: [customer_id#53 ASC NULLS FIRST], false, 0 (58) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#54] +Right keys [1]: [customer_id#53] Join condition: None (59) Project [codegen id : 26] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#55] -Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#54, year_total#55] +Output [7]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54] +Input [8]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, customer_id#53, year_total#54] (60) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +Output [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#59), dynamicpruningexpression(ws_sold_date_sk#59 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] (62) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] -Condition : isnotnull(ws_bill_customer_sk#57) +Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_customer_sk#56) (63) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#60, d_year#61] +Output [2]: [d_date_sk#59, d_year#60] (64) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#59] -Right keys [1]: [d_date_sk#60] +Left keys [1]: [ws_sold_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (65) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#57, ws_net_paid#58, d_year#61] -Input [5]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59, d_date_sk#60, d_year#61] +Output [3]: [ws_bill_customer_sk#56, ws_net_paid#57, d_year#60] +Input [5]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] (66) Exchange -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, d_year#61] -Arguments: hashpartitioning(ws_bill_customer_sk#57, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, d_year#60] +Arguments: hashpartitioning(ws_bill_customer_sk#56, 5), ENSURE_REQUIREMENTS, [id=#61] (67) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, d_year#61] -Arguments: [ws_bill_customer_sk#57 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, d_year#60] +Arguments: [ws_bill_customer_sk#56 ASC NULLS FIRST], false, 0 (68) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66] +Output [4]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65] (69) Sort [codegen id : 31] -Input [4]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66] -Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65] +Arguments: [c_customer_sk#62 ASC NULLS FIRST], false, 0 (70) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#57] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ws_bill_customer_sk#56] +Right keys [1]: [c_customer_sk#62] Join condition: None (71) Project [codegen id : 32] -Output [5]: [c_customer_id#64, c_first_name#65, c_last_name#66, ws_net_paid#58, d_year#61] -Input [7]: [ws_bill_customer_sk#57, ws_net_paid#58, d_year#61, c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66] +Output [5]: [c_customer_id#63, c_first_name#64, c_last_name#65, ws_net_paid#57, d_year#60] +Input [7]: [ws_bill_customer_sk#56, ws_net_paid#57, d_year#60, c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65] (72) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#64, c_first_name#65, c_last_name#66, ws_net_paid#58, d_year#61] -Keys [4]: [c_customer_id#64, c_first_name#65, c_last_name#66, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#58))] -Aggregate Attributes [1]: [sum#67] -Results [5]: [c_customer_id#64, c_first_name#65, c_last_name#66, d_year#61, sum#68] +Input [5]: [c_customer_id#63, c_first_name#64, c_last_name#65, ws_net_paid#57, d_year#60] +Keys [4]: [c_customer_id#63, c_first_name#64, c_last_name#65, d_year#60] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] +Aggregate Attributes [1]: [sum#66] +Results [5]: [c_customer_id#63, c_first_name#64, c_last_name#65, d_year#60, sum#67] (73) Exchange -Input [5]: [c_customer_id#64, c_first_name#65, c_last_name#66, d_year#61, sum#68] -Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [5]: [c_customer_id#63, c_first_name#64, c_last_name#65, d_year#60, sum#67] +Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, d_year#60, 5), ENSURE_REQUIREMENTS, [id=#68] (74) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#64, c_first_name#65, c_last_name#66, d_year#61, sum#68] -Keys [4]: [c_customer_id#64, c_first_name#65, c_last_name#66, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_net_paid#58))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#58))#70] -Results [2]: [c_customer_id#64 AS customer_id#71, MakeDecimal(sum(UnscaledValue(ws_net_paid#58))#70,17,2) AS year_total#72] +Input [5]: [c_customer_id#63, c_first_name#64, c_last_name#65, d_year#60, sum#67] +Keys [4]: [c_customer_id#63, c_first_name#64, c_last_name#65, d_year#60] +Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#57))#52] +Results [2]: [c_customer_id#63 AS customer_id#69, MakeDecimal(sum(UnscaledValue(ws_net_paid#57))#52,17,2) AS year_total#70] (75) Exchange -Input [2]: [customer_id#71, year_total#72] -Arguments: hashpartitioning(customer_id#71, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [2]: [customer_id#69, year_total#70] +Arguments: hashpartitioning(customer_id#69, 5), ENSURE_REQUIREMENTS, [id=#71] (76) Sort [codegen id : 34] -Input [2]: [customer_id#71, year_total#72] -Arguments: [customer_id#71 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#69, year_total#70] +Arguments: [customer_id#69 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#71] -Join condition: (CASE WHEN (year_total#55 > 0.00) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#38) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#69] +Join condition: (CASE WHEN (year_total#54 > 0.00) THEN CheckOverflow((promote_precision(year_total#70) / promote_precision(year_total#54)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#37) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (78) Project [codegen id : 35] -Output [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Input [9]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#55, customer_id#71, year_total#72] +Output [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] +Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54, customer_id#69, year_total#70] (79) TakeOrderedAndProject -Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Arguments: 100, [customer_first_name#36 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST, customer_last_name#37 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37] +Input [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] +Arguments: 100, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36] ===== Subqueries ===== @@ -463,7 +463,7 @@ Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2001)) AND d_year#6 IN (2001, (83) BroadcastExchange Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (87) @@ -488,10 +488,10 @@ Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (20 (87) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#59 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#23 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 7d73fade67751..7994ba12c2ac6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -217,195 +217,195 @@ Arguments: hashpartitioning(c_customer_id#19, c_first_name#20, c_last_name#21, d Input [5]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#28, sum#30] Keys [4]: [c_customer_id#19, c_first_name#20, c_last_name#21, d_year#28] Functions [1]: [sum(UnscaledValue(ss_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#23))#32] -Results [4]: [c_customer_id#19 AS customer_id#33, c_first_name#20 AS customer_first_name#34, c_last_name#21 AS customer_last_name#35, MakeDecimal(sum(UnscaledValue(ss_net_paid#23))#32,17,2) AS year_total#36] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#23))#15] +Results [4]: [c_customer_id#19 AS customer_id#32, c_first_name#20 AS customer_first_name#33, c_last_name#21 AS customer_last_name#34, MakeDecimal(sum(UnscaledValue(ss_net_paid#23))#15,17,2) AS year_total#35] (32) BroadcastExchange -Input [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] +Input [4]: [customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#36] (33) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#32] Join condition: None (34) Scan parquet default.customer -Output [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] +Output [4]: [c_customer_sk#37, c_customer_id#38, c_first_name#39, c_last_name#40] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] +Input [4]: [c_customer_sk#37, c_customer_id#38, c_first_name#39, c_last_name#40] (36) Filter [codegen id : 10] -Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Condition : (isnotnull(c_customer_sk#38) AND isnotnull(c_customer_id#39)) +Input [4]: [c_customer_sk#37, c_customer_id#38, c_first_name#39, c_last_name#40] +Condition : (isnotnull(c_customer_sk#37) AND isnotnull(c_customer_id#38)) (37) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Output [3]: [ws_bill_customer_sk#41, ws_net_paid#42, ws_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Input [3]: [ws_bill_customer_sk#41, ws_net_paid#42, ws_sold_date_sk#43] (39) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) +Input [3]: [ws_bill_customer_sk#41, ws_net_paid#42, ws_sold_date_sk#43] +Condition : isnotnull(ws_bill_customer_sk#41) (40) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +Input [3]: [ws_bill_customer_sk#41, ws_net_paid#42, ws_sold_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#38] -Right keys [1]: [ws_bill_customer_sk#42] +Left keys [1]: [c_customer_sk#37] +Right keys [1]: [ws_bill_customer_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Output [5]: [c_customer_id#38, c_first_name#39, c_last_name#40, ws_net_paid#42, ws_sold_date_sk#43] +Input [7]: [c_customer_sk#37, c_customer_id#38, c_first_name#39, c_last_name#40, ws_bill_customer_sk#41, ws_net_paid#42, ws_sold_date_sk#43] (43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#46, d_year#47] +Output [2]: [d_date_sk#45, d_year#46] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] +Left keys [1]: [ws_sold_date_sk#43] +Right keys [1]: [d_date_sk#45] Join condition: None (45) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] +Output [5]: [c_customer_id#38, c_first_name#39, c_last_name#40, ws_net_paid#42, d_year#46] +Input [7]: [c_customer_id#38, c_first_name#39, c_last_name#40, ws_net_paid#42, ws_sold_date_sk#43, d_date_sk#45, d_year#46] (46) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#48] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] +Input [5]: [c_customer_id#38, c_first_name#39, c_last_name#40, ws_net_paid#42, d_year#46] +Keys [4]: [c_customer_id#38, c_first_name#39, c_last_name#40, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#42))] +Aggregate Attributes [1]: [sum#47] +Results [5]: [c_customer_id#38, c_first_name#39, c_last_name#40, d_year#46, sum#48] (47) Exchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [c_customer_id#38, c_first_name#39, c_last_name#40, d_year#46, sum#48] +Arguments: hashpartitioning(c_customer_id#38, c_first_name#39, c_last_name#40, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#49] (48) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#51] -Results [2]: [c_customer_id#39 AS customer_id#52, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#51,17,2) AS year_total#53] +Input [5]: [c_customer_id#38, c_first_name#39, c_last_name#40, d_year#46, sum#48] +Keys [4]: [c_customer_id#38, c_first_name#39, c_last_name#40, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#42))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#42))#50] +Results [2]: [c_customer_id#38 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#42))#50,17,2) AS year_total#52] (49) Filter [codegen id : 11] -Input [2]: [customer_id#52, year_total#53] -Condition : (isnotnull(year_total#53) AND (year_total#53 > 0.00)) +Input [2]: [customer_id#51, year_total#52] +Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) (50) BroadcastExchange -Input [2]: [customer_id#52, year_total#53] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#54] +Input [2]: [customer_id#51, year_total#52] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#53] (51) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#52] +Right keys [1]: [customer_id#51] Join condition: None (52) Project [codegen id : 16] -Output [7]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, year_total#53] -Input [8]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, customer_id#52, year_total#53] +Output [7]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52] +Input [8]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, customer_id#51, year_total#52] (53) Scan parquet default.customer -Output [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] +Output [4]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (54) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] +Input [4]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57] (55) Filter [codegen id : 14] -Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(c_customer_id#56)) +Input [4]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57] +Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) (56) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] (58) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_customer_sk#58) (59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#55] -Right keys [1]: [ws_bill_customer_sk#59] +Left keys [1]: [c_customer_sk#54] +Right keys [1]: [ws_bill_customer_sk#58] Join condition: None (61) Project [codegen id : 14] -Output [5]: [c_customer_id#56, c_first_name#57, c_last_name#58, ws_net_paid#60, ws_sold_date_sk#61] -Input [7]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, ws_bill_customer_sk#59, ws_net_paid#60, ws_sold_date_sk#61] +Output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] +Input [7]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] (62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#63, d_year#64] +Output [2]: [d_date_sk#62, d_year#63] (63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#61] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#62] Join condition: None (64) Project [codegen id : 14] -Output [5]: [c_customer_id#56, c_first_name#57, c_last_name#58, ws_net_paid#60, d_year#64] -Input [7]: [c_customer_id#56, c_first_name#57, c_last_name#58, ws_net_paid#60, ws_sold_date_sk#61, d_date_sk#63, d_year#64] +Output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] +Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] (65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#56, c_first_name#57, c_last_name#58, ws_net_paid#60, d_year#64] -Keys [4]: [c_customer_id#56, c_first_name#57, c_last_name#58, d_year#64] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#60))] -Aggregate Attributes [1]: [sum#65] -Results [5]: [c_customer_id#56, c_first_name#57, c_last_name#58, d_year#64, sum#66] +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] +Aggregate Attributes [1]: [sum#64] +Results [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#65] (66) Exchange -Input [5]: [c_customer_id#56, c_first_name#57, c_last_name#58, d_year#64, sum#66] -Arguments: hashpartitioning(c_customer_id#56, c_first_name#57, c_last_name#58, d_year#64, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#65] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, [id=#66] (67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#56, c_first_name#57, c_last_name#58, d_year#64, sum#66] -Keys [4]: [c_customer_id#56, c_first_name#57, c_last_name#58, d_year#64] -Functions [1]: [sum(UnscaledValue(ws_net_paid#60))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#60))#68] -Results [2]: [c_customer_id#56 AS customer_id#69, MakeDecimal(sum(UnscaledValue(ws_net_paid#60))#68,17,2) AS year_total#70] +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#65] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#59))#50] +Results [2]: [c_customer_id#55 AS customer_id#67, MakeDecimal(sum(UnscaledValue(ws_net_paid#59))#50,17,2) AS year_total#68] (68) BroadcastExchange -Input [2]: [customer_id#69, year_total#70] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#71] +Input [2]: [customer_id#67, year_total#68] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#69] (69) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#69] -Join condition: (CASE WHEN (year_total#53 > 0.00) THEN CheckOverflow((promote_precision(year_total#70) / promote_precision(year_total#53)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#36) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#67] +Join condition: (CASE WHEN (year_total#52 > 0.00) THEN CheckOverflow((promote_precision(year_total#68) / promote_precision(year_total#52)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#17 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#17)), DecimalType(37,20), true) ELSE null END) (70) Project [codegen id : 16] -Output [3]: [customer_id#33, customer_first_name#34, customer_last_name#35] -Input [9]: [customer_id#16, year_total#17, customer_id#33, customer_first_name#34, customer_last_name#35, year_total#36, year_total#53, customer_id#69, year_total#70] +Output [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] +Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52, customer_id#67, year_total#68] (71) TakeOrderedAndProject -Input [3]: [customer_id#33, customer_first_name#34, customer_last_name#35] -Arguments: 100, [customer_first_name#34 ASC NULLS FIRST, customer_id#33 ASC NULLS FIRST, customer_last_name#35 ASC NULLS FIRST], [customer_id#33, customer_first_name#34, customer_last_name#35] +Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] +Arguments: 100, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] ===== Subqueries ===== @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2001)) AND d_year#11 IN (20 (75) BroadcastExchange Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (79) @@ -457,10 +457,10 @@ Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (20 (79) BroadcastExchange Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#71] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#25 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#25 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt index 00064ea21ec32..cd66823f10e8c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt @@ -652,40 +652,40 @@ Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manuf Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#23, sales_amt#24] Keys [5]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88] Functions [2]: [partial_sum(sales_cnt#23), partial_sum(UnscaledValue(sales_amt#24))] -Aggregate Attributes [2]: [sum#131, sum#132] -Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#133, sum#134] +Aggregate Attributes [2]: [sum#68, sum#131] +Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#70, sum#132] (121) Exchange -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#133, sum#134] -Arguments: hashpartitioning(d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#70, sum#132] +Arguments: hashpartitioning(d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#133] (122) HashAggregate [codegen id : 49] -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#133, sum#134] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#70, sum#132] Keys [5]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88] Functions [2]: [sum(sales_cnt#23), sum(UnscaledValue(sales_amt#24))] -Aggregate Attributes [2]: [sum(sales_cnt#23)#136, sum(UnscaledValue(sales_amt#24))#137] -Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum(sales_cnt#23)#136 AS sales_cnt#138, MakeDecimal(sum(UnscaledValue(sales_amt#24))#137,18,2) AS sales_amt#139] +Aggregate Attributes [2]: [sum(sales_cnt#23)#73, sum(UnscaledValue(sales_amt#24))#74] +Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum(sales_cnt#23)#73 AS sales_cnt#134, MakeDecimal(sum(UnscaledValue(sales_amt#24))#74,18,2) AS sales_amt#135] (123) Exchange -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#138, sales_amt#139] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#140] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#134, sales_amt#135] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#136] (124) Sort [codegen id : 50] -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#138, sales_amt#139] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#134, sales_amt#135] Arguments: [i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST, i_manufact_id#88 ASC NULLS FIRST], false, 0 (125) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#75 as decimal(17,2))) / promote_precision(cast(sales_cnt#138 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#75 as decimal(17,2))) / promote_precision(cast(sales_cnt#134 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (126) Project [codegen id : 51] -Output [10]: [d_year#90 AS prev_year#141, d_year#15 AS year#142, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#138 AS prev_yr_cnt#143, sales_cnt#75 AS curr_yr_cnt#144, (sales_cnt#75 - sales_cnt#138) AS sales_cnt_diff#145, CheckOverflow((promote_precision(cast(sales_amt#76 as decimal(19,2))) - promote_precision(cast(sales_amt#139 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#146] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#75, sales_amt#76, d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#138, sales_amt#139] +Output [10]: [d_year#90 AS prev_year#137, d_year#15 AS year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#134 AS prev_yr_cnt#139, sales_cnt#75 AS curr_yr_cnt#140, (sales_cnt#75 - sales_cnt#134) AS sales_cnt_diff#141, CheckOverflow((promote_precision(cast(sales_amt#76 as decimal(19,2))) - promote_precision(cast(sales_amt#135 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#142] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#75, sales_amt#76, d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#134, sales_amt#135] (127) TakeOrderedAndProject -Input [10]: [prev_year#141, year#142, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#143, curr_yr_cnt#144, sales_cnt_diff#145, sales_amt_diff#146] -Arguments: 100, [sales_cnt_diff#145 ASC NULLS FIRST, sales_amt_diff#146 ASC NULLS FIRST], [prev_year#141, year#142, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#143, curr_yr_cnt#144, sales_cnt_diff#145, sales_amt_diff#146] +Input [10]: [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] +Arguments: 100, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] ===== Subqueries ===== @@ -712,7 +712,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ (131) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#143] Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#6 @@ -741,7 +741,7 @@ Condition : ((isnotnull(d_year#90) AND (d_year#90 = 2001)) AND isnotnull(d_date_ (135) BroadcastExchange Input [2]: [d_date_sk#89, d_year#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#144] Subquery:5 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#83 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 00064ea21ec32..cd66823f10e8c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -652,40 +652,40 @@ Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manuf Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#23, sales_amt#24] Keys [5]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88] Functions [2]: [partial_sum(sales_cnt#23), partial_sum(UnscaledValue(sales_amt#24))] -Aggregate Attributes [2]: [sum#131, sum#132] -Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#133, sum#134] +Aggregate Attributes [2]: [sum#68, sum#131] +Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#70, sum#132] (121) Exchange -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#133, sum#134] -Arguments: hashpartitioning(d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#70, sum#132] +Arguments: hashpartitioning(d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#133] (122) HashAggregate [codegen id : 49] -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#133, sum#134] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum#70, sum#132] Keys [5]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88] Functions [2]: [sum(sales_cnt#23), sum(UnscaledValue(sales_amt#24))] -Aggregate Attributes [2]: [sum(sales_cnt#23)#136, sum(UnscaledValue(sales_amt#24))#137] -Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum(sales_cnt#23)#136 AS sales_cnt#138, MakeDecimal(sum(UnscaledValue(sales_amt#24))#137,18,2) AS sales_amt#139] +Aggregate Attributes [2]: [sum(sales_cnt#23)#73, sum(UnscaledValue(sales_amt#24))#74] +Results [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sum(sales_cnt#23)#73 AS sales_cnt#134, MakeDecimal(sum(UnscaledValue(sales_amt#24))#74,18,2) AS sales_amt#135] (123) Exchange -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#138, sales_amt#139] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#140] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#134, sales_amt#135] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, 5), ENSURE_REQUIREMENTS, [id=#136] (124) Sort [codegen id : 50] -Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#138, sales_amt#139] +Input [7]: [d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#134, sales_amt#135] Arguments: [i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST, i_manufact_id#88 ASC NULLS FIRST], false, 0 (125) SortMergeJoin [codegen id : 51] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#75 as decimal(17,2))) / promote_precision(cast(sales_cnt#138 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#75 as decimal(17,2))) / promote_precision(cast(sales_cnt#134 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (126) Project [codegen id : 51] -Output [10]: [d_year#90 AS prev_year#141, d_year#15 AS year#142, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#138 AS prev_yr_cnt#143, sales_cnt#75 AS curr_yr_cnt#144, (sales_cnt#75 - sales_cnt#138) AS sales_cnt_diff#145, CheckOverflow((promote_precision(cast(sales_amt#76 as decimal(19,2))) - promote_precision(cast(sales_amt#139 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#146] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#75, sales_amt#76, d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#138, sales_amt#139] +Output [10]: [d_year#90 AS prev_year#137, d_year#15 AS year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#134 AS prev_yr_cnt#139, sales_cnt#75 AS curr_yr_cnt#140, (sales_cnt#75 - sales_cnt#134) AS sales_cnt_diff#141, CheckOverflow((promote_precision(cast(sales_amt#76 as decimal(19,2))) - promote_precision(cast(sales_amt#135 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#142] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#75, sales_amt#76, d_year#90, i_brand_id#85, i_class_id#86, i_category_id#87, i_manufact_id#88, sales_cnt#134, sales_amt#135] (127) TakeOrderedAndProject -Input [10]: [prev_year#141, year#142, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#143, curr_yr_cnt#144, sales_cnt_diff#145, sales_amt_diff#146] -Arguments: 100, [sales_cnt_diff#145 ASC NULLS FIRST, sales_amt_diff#146 ASC NULLS FIRST], [prev_year#141, year#142, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#143, curr_yr_cnt#144, sales_cnt_diff#145, sales_amt_diff#146] +Input [10]: [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] +Arguments: 100, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] ===== Subqueries ===== @@ -712,7 +712,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ (131) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#143] Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#6 @@ -741,7 +741,7 @@ Condition : ((isnotnull(d_year#90) AND (d_year#90 = 2001)) AND isnotnull(d_date_ (135) BroadcastExchange Input [2]: [d_date_sk#89, d_year#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#144] Subquery:5 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#83 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt index 15643ed89e24c..4d27141fd8465 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt @@ -495,60 +495,60 @@ Aggregate Attributes [3]: [sum(sales#16)#118, sum(returns#36)#119, sum(profit#37 Results [5]: [channel#34, id#35, cast(sum(sales#16)#118 as decimal(37,2)) AS sales#121, cast(sum(returns#36)#119 as decimal(37,2)) AS returns#122, cast(sum(profit#37)#120 as decimal(38,2)) AS profit#123] (84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Output [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] (85) HashAggregate [codegen id : 48] -Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Input [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#16), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#16)#130, sum(returns#36)#131, sum(profit#37)#132] -Results [4]: [channel#34, sum(sales#16)#130 AS sales#133, sum(returns#36)#131 AS returns#134, sum(profit#37)#132 AS profit#135] +Aggregate Attributes [3]: [sum(sales#16)#118, sum(returns#36)#119, sum(profit#37)#120] +Results [4]: [channel#34, sum(sales#16)#118 AS sales#124, sum(returns#36)#119 AS returns#125, sum(profit#37)#120 AS profit#126] (86) HashAggregate [codegen id : 48] -Input [4]: [channel#34, sales#133, returns#134, profit#135] +Input [4]: [channel#34, sales#124, returns#125, profit#126] Keys [1]: [channel#34] -Functions [3]: [partial_sum(sales#133), partial_sum(returns#134), partial_sum(profit#135)] -Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Results [7]: [channel#34, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] +Aggregate Attributes [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Results [7]: [channel#34, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] (87) Exchange -Input [7]: [channel#34, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [id=#148] +Input [7]: [channel#34, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [id=#139] (88) HashAggregate [codegen id : 49] -Input [7]: [channel#34, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Input [7]: [channel#34, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Keys [1]: [channel#34] -Functions [3]: [sum(sales#133), sum(returns#134), sum(profit#135)] -Aggregate Attributes [3]: [sum(sales#133)#149, sum(returns#134)#150, sum(profit#135)#151] -Results [5]: [channel#34, null AS id#152, sum(sales#133)#149 AS sales#153, sum(returns#134)#150 AS returns#154, sum(profit#135)#151 AS profit#155] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] +Aggregate Attributes [3]: [sum(sales#124)#140, sum(returns#125)#141, sum(profit#126)#142] +Results [5]: [channel#34, null AS id#143, sum(sales#124)#140 AS sales#144, sum(returns#125)#141 AS returns#145, sum(profit#126)#142 AS profit#146] (89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#34, id#35, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Output [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] (90) HashAggregate [codegen id : 73] -Input [8]: [channel#34, id#35, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#16), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#16)#162, sum(returns#36)#163, sum(profit#37)#164] -Results [3]: [sum(sales#16)#162 AS sales#133, sum(returns#36)#163 AS returns#134, sum(profit#37)#164 AS profit#135] +Aggregate Attributes [3]: [sum(sales#16)#118, sum(returns#36)#119, sum(profit#37)#120] +Results [3]: [sum(sales#16)#118 AS sales#124, sum(returns#36)#119 AS returns#125, sum(profit#37)#120 AS profit#126] (91) HashAggregate [codegen id : 73] -Input [3]: [sales#133, returns#134, profit#135] +Input [3]: [sales#124, returns#125, profit#126] Keys: [] -Functions [3]: [partial_sum(sales#133), partial_sum(returns#134), partial_sum(profit#135)] -Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] +Aggregate Attributes [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Results [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] (92) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (93) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys: [] -Functions [3]: [sum(sales#133), sum(returns#134), sum(profit#135)] -Aggregate Attributes [3]: [sum(sales#133)#178, sum(returns#134)#179, sum(profit#135)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#133)#178 AS sales#183, sum(returns#134)#179 AS returns#184, sum(profit#135)#180 AS profit#185] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] +Aggregate Attributes [3]: [sum(sales#124)#160, sum(returns#125)#161, sum(profit#126)#162] +Results [5]: [null AS channel#163, null AS id#164, sum(sales#124)#160 AS sales#165, sum(returns#125)#161 AS returns#166, sum(profit#126)#162 AS profit#167] (94) Union @@ -561,7 +561,7 @@ Results [5]: [channel#34, id#35, sales#121, returns#122, profit#123] (96) Exchange Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123] -Arguments: hashpartitioning(channel#34, id#35, sales#121, returns#122, profit#123, 5), ENSURE_REQUIREMENTS, [id=#186] +Arguments: hashpartitioning(channel#34, id#35, sales#121, returns#122, profit#123, 5), ENSURE_REQUIREMENTS, [id=#168] (97) HashAggregate [codegen id : 76] Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123] @@ -585,26 +585,26 @@ BroadcastExchange (103) (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_date#187] +Output [2]: [d_date_sk#6, d_date#169] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#187] +Input [2]: [d_date_sk#6, d_date#169] (101) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#187] -Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_date#169] +Condition : (((isnotnull(d_date#169) AND (d_date#169 >= 1998-08-04)) AND (d_date#169 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) (102) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_date#187] +Input [2]: [d_date_sk#6, d_date#169] (103) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#188] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#170] Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#21 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index 63348f9a421f7..a1d99b72c8147 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -495,60 +495,60 @@ Aggregate Attributes [3]: [sum(sales#16)#118, sum(returns#36)#119, sum(profit#37 Results [5]: [channel#34, id#35, cast(sum(sales#16)#118 as decimal(37,2)) AS sales#121, cast(sum(returns#36)#119 as decimal(37,2)) AS returns#122, cast(sum(profit#37)#120 as decimal(38,2)) AS profit#123] (84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Output [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] (85) HashAggregate [codegen id : 48] -Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Input [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#16), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#16)#130, sum(returns#36)#131, sum(profit#37)#132] -Results [4]: [channel#34, sum(sales#16)#130 AS sales#133, sum(returns#36)#131 AS returns#134, sum(profit#37)#132 AS profit#135] +Aggregate Attributes [3]: [sum(sales#16)#118, sum(returns#36)#119, sum(profit#37)#120] +Results [4]: [channel#34, sum(sales#16)#118 AS sales#124, sum(returns#36)#119 AS returns#125, sum(profit#37)#120 AS profit#126] (86) HashAggregate [codegen id : 48] -Input [4]: [channel#34, sales#133, returns#134, profit#135] +Input [4]: [channel#34, sales#124, returns#125, profit#126] Keys [1]: [channel#34] -Functions [3]: [partial_sum(sales#133), partial_sum(returns#134), partial_sum(profit#135)] -Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Results [7]: [channel#34, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] +Aggregate Attributes [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Results [7]: [channel#34, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] (87) Exchange -Input [7]: [channel#34, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [id=#148] +Input [7]: [channel#34, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [id=#139] (88) HashAggregate [codegen id : 49] -Input [7]: [channel#34, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Input [7]: [channel#34, sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] Keys [1]: [channel#34] -Functions [3]: [sum(sales#133), sum(returns#134), sum(profit#135)] -Aggregate Attributes [3]: [sum(sales#133)#149, sum(returns#134)#150, sum(profit#135)#151] -Results [5]: [channel#34, null AS id#152, sum(sales#133)#149 AS sales#153, sum(returns#134)#150 AS returns#154, sum(profit#135)#151 AS profit#155] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] +Aggregate Attributes [3]: [sum(sales#124)#140, sum(returns#125)#141, sum(profit#126)#142] +Results [5]: [channel#34, null AS id#143, sum(sales#124)#140 AS sales#144, sum(returns#125)#141 AS returns#145, sum(profit#126)#142 AS profit#146] (89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#34, id#35, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Output [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] (90) HashAggregate [codegen id : 73] -Input [8]: [channel#34, id#35, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [8]: [channel#34, id#35, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] Keys [2]: [channel#34, id#35] Functions [3]: [sum(sales#16), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#16)#162, sum(returns#36)#163, sum(profit#37)#164] -Results [3]: [sum(sales#16)#162 AS sales#133, sum(returns#36)#163 AS returns#134, sum(profit#37)#164 AS profit#135] +Aggregate Attributes [3]: [sum(sales#16)#118, sum(returns#36)#119, sum(profit#37)#120] +Results [3]: [sum(sales#16)#118 AS sales#124, sum(returns#36)#119 AS returns#125, sum(profit#37)#120 AS profit#126] (91) HashAggregate [codegen id : 73] -Input [3]: [sales#133, returns#134, profit#135] +Input [3]: [sales#124, returns#125, profit#126] Keys: [] -Functions [3]: [partial_sum(sales#133), partial_sum(returns#134), partial_sum(profit#135)] -Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] +Aggregate Attributes [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Results [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] (92) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (93) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys: [] -Functions [3]: [sum(sales#133), sum(returns#134), sum(profit#135)] -Aggregate Attributes [3]: [sum(sales#133)#178, sum(returns#134)#179, sum(profit#135)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#133)#178 AS sales#183, sum(returns#134)#179 AS returns#184, sum(profit#135)#180 AS profit#185] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] +Aggregate Attributes [3]: [sum(sales#124)#160, sum(returns#125)#161, sum(profit#126)#162] +Results [5]: [null AS channel#163, null AS id#164, sum(sales#124)#160 AS sales#165, sum(returns#125)#161 AS returns#166, sum(profit#126)#162 AS profit#167] (94) Union @@ -561,7 +561,7 @@ Results [5]: [channel#34, id#35, sales#121, returns#122, profit#123] (96) Exchange Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123] -Arguments: hashpartitioning(channel#34, id#35, sales#121, returns#122, profit#123, 5), ENSURE_REQUIREMENTS, [id=#186] +Arguments: hashpartitioning(channel#34, id#35, sales#121, returns#122, profit#123, 5), ENSURE_REQUIREMENTS, [id=#168] (97) HashAggregate [codegen id : 76] Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123] @@ -585,26 +585,26 @@ BroadcastExchange (103) (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_date#187] +Output [2]: [d_date_sk#6, d_date#169] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#187] +Input [2]: [d_date_sk#6, d_date#169] (101) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_date#187] -Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_date#169] +Condition : (((isnotnull(d_date#169) AND (d_date#169 >= 1998-08-04)) AND (d_date#169 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) (102) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_date#187] +Input [2]: [d_date_sk#6, d_date#169] (103) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#188] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#170] Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#21 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index a515ff13e1c12..34777c108a268 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -597,60 +597,60 @@ Aggregate Attributes [3]: [sum(sales#42)#136, sum(returns#43)#137, sum(profit#44 Results [5]: [channel#40, id#41, cast(sum(sales#42)#136 as decimal(37,2)) AS sales#139, cast(sum(returns#43)#137 as decimal(38,2)) AS returns#140, cast(sum(profit#44)#138 as decimal(38,2)) AS profit#141] (106) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#40, id#41, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Output [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] (107) HashAggregate [codegen id : 64] -Input [8]: [channel#40, id#41, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Input [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Keys [2]: [channel#40, id#41] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#148, sum(returns#43)#149, sum(profit#44)#150] -Results [4]: [channel#40, sum(sales#42)#148 AS sales#151, sum(returns#43)#149 AS returns#152, sum(profit#44)#150 AS profit#153] +Aggregate Attributes [3]: [sum(sales#42)#136, sum(returns#43)#137, sum(profit#44)#138] +Results [4]: [channel#40, sum(sales#42)#136 AS sales#142, sum(returns#43)#137 AS returns#143, sum(profit#44)#138 AS profit#144] (108) HashAggregate [codegen id : 64] -Input [4]: [channel#40, sales#151, returns#152, profit#153] +Input [4]: [channel#40, sales#142, returns#143, profit#144] Keys [1]: [channel#40] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Results [7]: [channel#40, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Functions [3]: [partial_sum(sales#142), partial_sum(returns#143), partial_sum(profit#144)] +Aggregate Attributes [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Results [7]: [channel#40, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] (109) Exchange -Input [7]: [channel#40, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Arguments: hashpartitioning(channel#40, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [7]: [channel#40, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] +Arguments: hashpartitioning(channel#40, 5), ENSURE_REQUIREMENTS, [id=#157] (110) HashAggregate [codegen id : 65] -Input [7]: [channel#40, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Input [7]: [channel#40, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] Keys [1]: [channel#40] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#167, sum(returns#152)#168, sum(profit#153)#169] -Results [5]: [channel#40, null AS id#170, sum(sales#151)#167 AS sales#171, sum(returns#152)#168 AS returns#172, sum(profit#153)#169 AS profit#173] +Functions [3]: [sum(sales#142), sum(returns#143), sum(profit#144)] +Aggregate Attributes [3]: [sum(sales#142)#158, sum(returns#143)#159, sum(profit#144)#160] +Results [5]: [channel#40, null AS id#161, sum(sales#142)#158 AS sales#162, sum(returns#143)#159 AS returns#163, sum(profit#144)#160 AS profit#164] (111) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#40, id#41, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] +Output [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] (112) HashAggregate [codegen id : 97] -Input [8]: [channel#40, id#41, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] +Input [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Keys [2]: [channel#40, id#41] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#180, sum(returns#43)#181, sum(profit#44)#182] -Results [3]: [sum(sales#42)#180 AS sales#151, sum(returns#43)#181 AS returns#152, sum(profit#44)#182 AS profit#153] +Aggregate Attributes [3]: [sum(sales#42)#136, sum(returns#43)#137, sum(profit#44)#138] +Results [3]: [sum(sales#42)#136 AS sales#142, sum(returns#43)#137 AS returns#143, sum(profit#44)#138 AS profit#144] (113) HashAggregate [codegen id : 97] -Input [3]: [sales#151, returns#152, profit#153] +Input [3]: [sales#142, returns#143, profit#144] Keys: [] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -Results [6]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Functions [3]: [partial_sum(sales#142), partial_sum(returns#143), partial_sum(profit#144)] +Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] (114) Exchange -Input [6]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#195] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] (115) HashAggregate [codegen id : 98] -Input [6]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#196, sum(returns#152)#197, sum(profit#153)#198] -Results [5]: [null AS channel#199, null AS id#200, sum(sales#151)#196 AS sales#201, sum(returns#152)#197 AS returns#202, sum(profit#153)#198 AS profit#203] +Functions [3]: [sum(sales#142), sum(returns#143), sum(profit#144)] +Aggregate Attributes [3]: [sum(sales#142)#178, sum(returns#143)#179, sum(profit#144)#180] +Results [5]: [null AS channel#181, null AS id#182, sum(sales#142)#178 AS sales#183, sum(returns#143)#179 AS returns#184, sum(profit#144)#180 AS profit#185] (116) Union @@ -663,7 +663,7 @@ Results [5]: [channel#40, id#41, sales#139, returns#140, profit#141] (118) Exchange Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141] -Arguments: hashpartitioning(channel#40, id#41, sales#139, returns#140, profit#141, 5), ENSURE_REQUIREMENTS, [id=#204] +Arguments: hashpartitioning(channel#40, id#41, sales#139, returns#140, profit#141, 5), ENSURE_REQUIREMENTS, [id=#186] (119) HashAggregate [codegen id : 100] Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141] @@ -687,26 +687,26 @@ BroadcastExchange (125) (121) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#205] +Output [2]: [d_date_sk#22, d_date#187] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#205] +Input [2]: [d_date_sk#22, d_date#187] (123) Filter [codegen id : 1] -Input [2]: [d_date_sk#22, d_date#205] -Condition : (((isnotnull(d_date#205) AND (d_date#205 >= 1998-08-04)) AND (d_date#205 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#187] +Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) (124) Project [codegen id : 1] Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#205] +Input [2]: [d_date_sk#22, d_date#187] (125) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#206] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#188] Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index ec24e90dd3c5b..3e68f3fe694fc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -597,60 +597,60 @@ Aggregate Attributes [3]: [sum(sales#42)#136, sum(returns#43)#137, sum(profit#44 Results [5]: [channel#40, id#41, cast(sum(sales#42)#136 as decimal(37,2)) AS sales#139, cast(sum(returns#43)#137 as decimal(38,2)) AS returns#140, cast(sum(profit#44)#138 as decimal(38,2)) AS profit#141] (106) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#40, id#41, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Output [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] (107) HashAggregate [codegen id : 64] -Input [8]: [channel#40, id#41, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Input [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Keys [2]: [channel#40, id#41] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#148, sum(returns#43)#149, sum(profit#44)#150] -Results [4]: [channel#40, sum(sales#42)#148 AS sales#151, sum(returns#43)#149 AS returns#152, sum(profit#44)#150 AS profit#153] +Aggregate Attributes [3]: [sum(sales#42)#136, sum(returns#43)#137, sum(profit#44)#138] +Results [4]: [channel#40, sum(sales#42)#136 AS sales#142, sum(returns#43)#137 AS returns#143, sum(profit#44)#138 AS profit#144] (108) HashAggregate [codegen id : 64] -Input [4]: [channel#40, sales#151, returns#152, profit#153] +Input [4]: [channel#40, sales#142, returns#143, profit#144] Keys [1]: [channel#40] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#154, isEmpty#155, sum#156, isEmpty#157, sum#158, isEmpty#159] -Results [7]: [channel#40, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Functions [3]: [partial_sum(sales#142), partial_sum(returns#143), partial_sum(profit#144)] +Aggregate Attributes [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Results [7]: [channel#40, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] (109) Exchange -Input [7]: [channel#40, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Arguments: hashpartitioning(channel#40, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [7]: [channel#40, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] +Arguments: hashpartitioning(channel#40, 5), ENSURE_REQUIREMENTS, [id=#157] (110) HashAggregate [codegen id : 65] -Input [7]: [channel#40, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Input [7]: [channel#40, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] Keys [1]: [channel#40] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#167, sum(returns#152)#168, sum(profit#153)#169] -Results [5]: [channel#40, null AS id#170, sum(sales#151)#167 AS sales#171, sum(returns#152)#168 AS returns#172, sum(profit#153)#169 AS profit#173] +Functions [3]: [sum(sales#142), sum(returns#143), sum(profit#144)] +Aggregate Attributes [3]: [sum(sales#142)#158, sum(returns#143)#159, sum(profit#144)#160] +Results [5]: [channel#40, null AS id#161, sum(sales#142)#158 AS sales#162, sum(returns#143)#159 AS returns#163, sum(profit#144)#160 AS profit#164] (111) ReusedExchange [Reuses operator id: 104] -Output [8]: [channel#40, id#41, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] +Output [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] (112) HashAggregate [codegen id : 97] -Input [8]: [channel#40, id#41, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] +Input [8]: [channel#40, id#41, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] Keys [2]: [channel#40, id#41] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#180, sum(returns#43)#181, sum(profit#44)#182] -Results [3]: [sum(sales#42)#180 AS sales#151, sum(returns#43)#181 AS returns#152, sum(profit#44)#182 AS profit#153] +Aggregate Attributes [3]: [sum(sales#42)#136, sum(returns#43)#137, sum(profit#44)#138] +Results [3]: [sum(sales#42)#136 AS sales#142, sum(returns#43)#137 AS returns#143, sum(profit#44)#138 AS profit#144] (113) HashAggregate [codegen id : 97] -Input [3]: [sales#151, returns#152, profit#153] +Input [3]: [sales#142, returns#143, profit#144] Keys: [] -Functions [3]: [partial_sum(sales#151), partial_sum(returns#152), partial_sum(profit#153)] -Aggregate Attributes [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -Results [6]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Functions [3]: [partial_sum(sales#142), partial_sum(returns#143), partial_sum(profit#144)] +Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] (114) Exchange -Input [6]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#195] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] (115) HashAggregate [codegen id : 98] -Input [6]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#151), sum(returns#152), sum(profit#153)] -Aggregate Attributes [3]: [sum(sales#151)#196, sum(returns#152)#197, sum(profit#153)#198] -Results [5]: [null AS channel#199, null AS id#200, sum(sales#151)#196 AS sales#201, sum(returns#152)#197 AS returns#202, sum(profit#153)#198 AS profit#203] +Functions [3]: [sum(sales#142), sum(returns#143), sum(profit#144)] +Aggregate Attributes [3]: [sum(sales#142)#178, sum(returns#143)#179, sum(profit#144)#180] +Results [5]: [null AS channel#181, null AS id#182, sum(sales#142)#178 AS sales#183, sum(returns#143)#179 AS returns#184, sum(profit#144)#180 AS profit#185] (116) Union @@ -663,7 +663,7 @@ Results [5]: [channel#40, id#41, sales#139, returns#140, profit#141] (118) Exchange Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141] -Arguments: hashpartitioning(channel#40, id#41, sales#139, returns#140, profit#141, 5), ENSURE_REQUIREMENTS, [id=#204] +Arguments: hashpartitioning(channel#40, id#41, sales#139, returns#140, profit#141, 5), ENSURE_REQUIREMENTS, [id=#186] (119) HashAggregate [codegen id : 100] Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141] @@ -687,26 +687,26 @@ BroadcastExchange (125) (121) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_date#205] +Output [2]: [d_date_sk#16, d_date#187] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#16, d_date#205] +Input [2]: [d_date_sk#16, d_date#187] (123) Filter [codegen id : 1] -Input [2]: [d_date_sk#16, d_date#205] -Condition : (((isnotnull(d_date#205) AND (d_date#205 >= 1998-08-04)) AND (d_date#205 <= 1998-09-03)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_date#187] +Condition : (((isnotnull(d_date#187) AND (d_date#187 >= 1998-08-04)) AND (d_date#187 <= 1998-09-03)) AND isnotnull(d_date_sk#16)) (124) Project [codegen id : 1] Output [1]: [d_date_sk#16] -Input [2]: [d_date_sk#16, d_date#205] +Input [2]: [d_date_sk#16, d_date#187] (125) BroadcastExchange Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#206] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#188] Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt index 18c6d7ae287fd..1dd3dc76a8f56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt @@ -108,60 +108,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#8, i_class#7, 0 AS g_category#15, 0 AS g_class#16, 0 AS lochierarchy#17] (16) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#18, i_class#19, sum#20] +Output [3]: [i_category#8, i_class#7, sum#18] (17) HashAggregate [codegen id : 8] -Input [3]: [i_category#18, i_class#19, sum#20] -Keys [2]: [i_category#18, i_class#19] -Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#22] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#22,17,2) AS total_sum#23, i_category#18] +Input [3]: [i_category#8, i_class#7, sum#18] +Keys [2]: [i_category#8, i_class#7] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) AS total_sum#19, i_category#8] (18) HashAggregate [codegen id : 8] -Input [2]: [total_sum#23, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_sum(total_sum#23)] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [3]: [i_category#18, sum#26, isEmpty#27] +Input [2]: [total_sum#19, i_category#8] +Keys [1]: [i_category#8] +Functions [1]: [partial_sum(total_sum#19)] +Aggregate Attributes [2]: [sum#20, isEmpty#21] +Results [3]: [i_category#8, sum#22, isEmpty#23] (19) Exchange -Input [3]: [i_category#18, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_category#8, sum#22, isEmpty#23] +Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, [id=#24] (20) HashAggregate [codegen id : 9] -Input [3]: [i_category#18, sum#26, isEmpty#27] -Keys [1]: [i_category#18] -Functions [1]: [sum(total_sum#23)] -Aggregate Attributes [1]: [sum(total_sum#23)#29] -Results [6]: [sum(total_sum#23)#29 AS total_sum#30, i_category#18, null AS i_class#31, 0 AS g_category#32, 1 AS g_class#33, 1 AS lochierarchy#34] +Input [3]: [i_category#8, sum#22, isEmpty#23] +Keys [1]: [i_category#8] +Functions [1]: [sum(total_sum#19)] +Aggregate Attributes [1]: [sum(total_sum#19)#25] +Results [6]: [sum(total_sum#19)#25 AS total_sum#26, i_category#8, null AS i_class#27, 0 AS g_category#28, 1 AS g_class#29, 1 AS lochierarchy#30] (21) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#35, i_class#36, sum#37] +Output [3]: [i_category#8, i_class#7, sum#31] (22) HashAggregate [codegen id : 13] -Input [3]: [i_category#35, i_class#36, sum#37] -Keys [2]: [i_category#35, i_class#36] -Functions [1]: [sum(UnscaledValue(ws_net_paid#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#38))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#38))#39,17,2) AS total_sum#23] +Input [3]: [i_category#8, i_class#7, sum#31] +Keys [2]: [i_category#8, i_class#7] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) AS total_sum#19] (23) HashAggregate [codegen id : 13] -Input [1]: [total_sum#23] +Input [1]: [total_sum#19] Keys: [] -Functions [1]: [partial_sum(total_sum#23)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [2]: [sum#42, isEmpty#43] +Functions [1]: [partial_sum(total_sum#19)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [2]: [sum#34, isEmpty#35] (24) Exchange -Input [2]: [sum#42, isEmpty#43] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#44] +Input [2]: [sum#34, isEmpty#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] (25) HashAggregate [codegen id : 14] -Input [2]: [sum#42, isEmpty#43] +Input [2]: [sum#34, isEmpty#35] Keys: [] -Functions [1]: [sum(total_sum#23)] -Aggregate Attributes [1]: [sum(total_sum#23)#45] -Results [6]: [sum(total_sum#23)#45 AS total_sum#46, null AS i_category#47, null AS i_class#48, 1 AS g_category#49, 1 AS g_class#50, 2 AS lochierarchy#51] +Functions [1]: [sum(total_sum#19)] +Aggregate Attributes [1]: [sum(total_sum#19)#37] +Results [6]: [sum(total_sum#19)#37 AS total_sum#38, null AS i_category#39, null AS i_class#40, 1 AS g_category#41, 1 AS g_class#42, 2 AS lochierarchy#43] (26) Union @@ -174,34 +174,34 @@ Results [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, (28) Exchange Input [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17] -Arguments: hashpartitioning(total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17, 5), ENSURE_REQUIREMENTS, [id=#52] +Arguments: hashpartitioning(total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17, 5), ENSURE_REQUIREMENTS, [id=#44] (29) HashAggregate [codegen id : 16] Input [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17] Keys [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, CASE WHEN (g_class#16 = 0) THEN i_category#8 END AS _w0#53] +Results [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, CASE WHEN (g_class#16 = 0) THEN i_category#8 END AS _w0#45] (30) Exchange -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53] -Arguments: hashpartitioning(lochierarchy#17, _w0#53, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45] +Arguments: hashpartitioning(lochierarchy#17, _w0#45, 5), ENSURE_REQUIREMENTS, [id=#46] (31) Sort [codegen id : 17] -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53] -Arguments: [lochierarchy#17 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45] +Arguments: [lochierarchy#17 ASC NULLS FIRST, _w0#45 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST], false, 0 (32) Window -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#17, _w0#53, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#55], [lochierarchy#17, _w0#53], [total_sum#14 DESC NULLS LAST] +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45] +Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#17, _w0#45, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#47], [lochierarchy#17, _w0#45], [total_sum#14 DESC NULLS LAST] (33) Project [codegen id : 18] -Output [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#55] -Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53, rank_within_parent#55] +Output [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] +Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45, rank_within_parent#47] (34) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#55] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#55 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#55] +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] ===== Subqueries ===== @@ -214,25 +214,25 @@ BroadcastExchange (39) (35) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#56] +Output [2]: [d_date_sk#5, d_month_seq#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#56] +Input [2]: [d_date_sk#5, d_month_seq#48] (37) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#56] -Condition : (((isnotnull(d_month_seq#56) AND (d_month_seq#56 >= 1212)) AND (d_month_seq#56 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#48] +Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1212)) AND (d_month_seq#48 <= 1223)) AND isnotnull(d_date_sk#5)) (38) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_month_seq#56] +Input [2]: [d_date_sk#5, d_month_seq#48] (39) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index 18c6d7ae287fd..1dd3dc76a8f56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -108,60 +108,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#8, i_class#7, 0 AS g_category#15, 0 AS g_class#16, 0 AS lochierarchy#17] (16) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#18, i_class#19, sum#20] +Output [3]: [i_category#8, i_class#7, sum#18] (17) HashAggregate [codegen id : 8] -Input [3]: [i_category#18, i_class#19, sum#20] -Keys [2]: [i_category#18, i_class#19] -Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#22] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#22,17,2) AS total_sum#23, i_category#18] +Input [3]: [i_category#8, i_class#7, sum#18] +Keys [2]: [i_category#8, i_class#7] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) AS total_sum#19, i_category#8] (18) HashAggregate [codegen id : 8] -Input [2]: [total_sum#23, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_sum(total_sum#23)] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [3]: [i_category#18, sum#26, isEmpty#27] +Input [2]: [total_sum#19, i_category#8] +Keys [1]: [i_category#8] +Functions [1]: [partial_sum(total_sum#19)] +Aggregate Attributes [2]: [sum#20, isEmpty#21] +Results [3]: [i_category#8, sum#22, isEmpty#23] (19) Exchange -Input [3]: [i_category#18, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_category#8, sum#22, isEmpty#23] +Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, [id=#24] (20) HashAggregate [codegen id : 9] -Input [3]: [i_category#18, sum#26, isEmpty#27] -Keys [1]: [i_category#18] -Functions [1]: [sum(total_sum#23)] -Aggregate Attributes [1]: [sum(total_sum#23)#29] -Results [6]: [sum(total_sum#23)#29 AS total_sum#30, i_category#18, null AS i_class#31, 0 AS g_category#32, 1 AS g_class#33, 1 AS lochierarchy#34] +Input [3]: [i_category#8, sum#22, isEmpty#23] +Keys [1]: [i_category#8] +Functions [1]: [sum(total_sum#19)] +Aggregate Attributes [1]: [sum(total_sum#19)#25] +Results [6]: [sum(total_sum#19)#25 AS total_sum#26, i_category#8, null AS i_class#27, 0 AS g_category#28, 1 AS g_class#29, 1 AS lochierarchy#30] (21) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#35, i_class#36, sum#37] +Output [3]: [i_category#8, i_class#7, sum#31] (22) HashAggregate [codegen id : 13] -Input [3]: [i_category#35, i_class#36, sum#37] -Keys [2]: [i_category#35, i_class#36] -Functions [1]: [sum(UnscaledValue(ws_net_paid#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#38))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#38))#39,17,2) AS total_sum#23] +Input [3]: [i_category#8, i_class#7, sum#31] +Keys [2]: [i_category#8, i_class#7] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) AS total_sum#19] (23) HashAggregate [codegen id : 13] -Input [1]: [total_sum#23] +Input [1]: [total_sum#19] Keys: [] -Functions [1]: [partial_sum(total_sum#23)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [2]: [sum#42, isEmpty#43] +Functions [1]: [partial_sum(total_sum#19)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [2]: [sum#34, isEmpty#35] (24) Exchange -Input [2]: [sum#42, isEmpty#43] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#44] +Input [2]: [sum#34, isEmpty#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] (25) HashAggregate [codegen id : 14] -Input [2]: [sum#42, isEmpty#43] +Input [2]: [sum#34, isEmpty#35] Keys: [] -Functions [1]: [sum(total_sum#23)] -Aggregate Attributes [1]: [sum(total_sum#23)#45] -Results [6]: [sum(total_sum#23)#45 AS total_sum#46, null AS i_category#47, null AS i_class#48, 1 AS g_category#49, 1 AS g_class#50, 2 AS lochierarchy#51] +Functions [1]: [sum(total_sum#19)] +Aggregate Attributes [1]: [sum(total_sum#19)#37] +Results [6]: [sum(total_sum#19)#37 AS total_sum#38, null AS i_category#39, null AS i_class#40, 1 AS g_category#41, 1 AS g_class#42, 2 AS lochierarchy#43] (26) Union @@ -174,34 +174,34 @@ Results [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, (28) Exchange Input [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17] -Arguments: hashpartitioning(total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17, 5), ENSURE_REQUIREMENTS, [id=#52] +Arguments: hashpartitioning(total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17, 5), ENSURE_REQUIREMENTS, [id=#44] (29) HashAggregate [codegen id : 16] Input [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17] Keys [6]: [total_sum#14, i_category#8, i_class#7, g_category#15, g_class#16, lochierarchy#17] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, CASE WHEN (g_class#16 = 0) THEN i_category#8 END AS _w0#53] +Results [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, CASE WHEN (g_class#16 = 0) THEN i_category#8 END AS _w0#45] (30) Exchange -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53] -Arguments: hashpartitioning(lochierarchy#17, _w0#53, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45] +Arguments: hashpartitioning(lochierarchy#17, _w0#45, 5), ENSURE_REQUIREMENTS, [id=#46] (31) Sort [codegen id : 17] -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53] -Arguments: [lochierarchy#17 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45] +Arguments: [lochierarchy#17 ASC NULLS FIRST, _w0#45 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST], false, 0 (32) Window -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#17, _w0#53, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#55], [lochierarchy#17, _w0#53], [total_sum#14 DESC NULLS LAST] +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45] +Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#17, _w0#45, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#47], [lochierarchy#17, _w0#45], [total_sum#14 DESC NULLS LAST] (33) Project [codegen id : 18] -Output [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#55] -Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#53, rank_within_parent#55] +Output [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] +Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45, rank_within_parent#47] (34) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#55] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#55 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#55] +Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] ===== Subqueries ===== @@ -214,25 +214,25 @@ BroadcastExchange (39) (35) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#56] +Output [2]: [d_date_sk#5, d_month_seq#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#56] +Input [2]: [d_date_sk#5, d_month_seq#48] (37) Filter [codegen id : 1] -Input [2]: [d_date_sk#5, d_month_seq#56] -Condition : (((isnotnull(d_month_seq#56) AND (d_month_seq#56 >= 1212)) AND (d_month_seq#56 <= 1223)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#48] +Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1212)) AND (d_month_seq#48 <= 1223)) AND isnotnull(d_date_sk#5)) (38) Project [codegen id : 1] Output [1]: [d_date_sk#5] -Input [2]: [d_date_sk#5, d_month_seq#56] +Input [2]: [d_date_sk#5, d_month_seq#48] (39) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala new file mode 100644 index 0000000000000..7ee533ac26d2b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.plans.logical.WithCTE +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +abstract class CTEInlineSuiteBase + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + + import testImplicits._ + + test("SPARK-36447: non-deterministic CTE dedup") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v as ( + | select c1, c2, rand() from t + |) + |select * from v except select * from v + """.stripMargin) + checkAnswer(df, Nil) + assert( + df.queryExecution.optimizedPlan.find(_.isInstanceOf[WithCTE]).nonEmpty, + "Non-deterministic With-CTE with multiple references should be not inlined.") + } + } + + test("SPARK-36447: non-deterministic CTE in subquery") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v as ( + | select c1, c2, rand() c3 from t + |) + |select * from v where c3 not in (select c3 from v) + """.stripMargin) + checkAnswer(df, Nil) + assert( + df.queryExecution.optimizedPlan.find(_.isInstanceOf[WithCTE]).nonEmpty, + "Non-deterministic With-CTE with multiple references should be not inlined.") + } + } + + test("SPARK-36447: non-deterministic CTE with one reference should be inlined") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v as ( + | select c1, c2, rand() c3 from t + |) + |select c1, c2 from v where c3 > 0 + """.stripMargin) + checkAnswer(df, Row(0, 1) :: Row(1, 2) :: Nil) + assert( + df.queryExecution.analyzed.find(_.isInstanceOf[WithCTE]).nonEmpty, + "With-CTE should not be inlined in analyzed plan.") + assert( + df.queryExecution.optimizedPlan.find(_.isInstanceOf[WithCTE]).isEmpty, + "With-CTE with one reference should be inlined in optimized plan.") + } + } + + test("SPARK-36447: nested non-deterministic CTEs referenced more than once are not inlined") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v1 as ( + | select c1, c2, rand() c3 from t + |), + |v2 as ( + | select c1, c2, rand() c4 from v1 where c3 in (select c3 from v1) + |) + |select count(*) from ( + | select * from v2 where c1 > 0 union select * from v2 where c2 > 0 + |) + """.stripMargin) + checkAnswer(df, Row(2) :: Nil) + assert( + df.queryExecution.analyzed.collect { + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 2, + "With-CTE should contain 2 CTE defs after analysis.") + assert( + df.queryExecution.optimizedPlan.collect { + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 2, + "With-CTE should contain 2 CTE def after optimization.") + } + } + + test("SPARK-36447: nested CTEs only the deterministic is inlined") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v1 as ( + | select c1, c2, rand() c3 from t + |), + |v2 as ( + | select * from v1 where c3 in (select c3 from v1) + |) + |select count(*) from ( + | select * from v2 where c1 > 0 union select * from v2 where c2 > 0 + |) + """.stripMargin) + checkAnswer(df, Row(2) :: Nil) + assert( + df.queryExecution.analyzed.collect { + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 2, + "With-CTE should contain 2 CTE defs after analysis.") + assert( + df.queryExecution.optimizedPlan.collect { + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 1, + "One CTE def should be inlined after optimization.") + } + } + + test("SPARK-36447: nested non-deterministic CTEs referenced only once are inlined") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v1 as ( + | select c1, c2, rand() c3 from t + |), + |v2 as ( + | select c1, c2, c3, rand() c4 from v1 + |) + |select c1, c2 from v2 where c3 > 0 and c4 > 0 + """.stripMargin) + checkAnswer(df, Row(0, 1) :: Row(1, 2) :: Nil) + assert( + df.queryExecution.analyzed.collect { + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 2, + "With-CTE should contain 2 CTE defs after analysis.") + assert( + df.queryExecution.optimizedPlan.collect { + case WithCTE(_, cteDefs) => cteDefs + }.isEmpty, + "CTEs with one reference should all be inlined after optimization.") + } + } + + test("SPARK-36447: With in subquery of main query") { + withSQLConf( + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName) { + withView("t") { + Seq((2, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with v as ( + | select c1, c2, rand() c3 from t + |) + |select * from v except + |select * from v where c1 = ( + | with v2 as ( + | select c1, c2, rand() c3 from t + | ) + | select count(*) from v where c2 not in ( + | select c2 from v2 where c3 not in (select c3 from v2) + | ) + |) + """.stripMargin) + checkAnswer(df, Nil) + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case r: ReusedExchangeExec => r + }.length == 3, + "Non-deterministic CTEs are reused shuffles.") + } + } + } + + test("SPARK-36447: With in subquery of CTE def") { + withView("t") { + Seq((2, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with v as ( + | select c1, c2, rand() c3 from t where c1 = ( + | with v2 as ( + | select c1, c2, rand() c3 from t + | ) + | select count(*) from ( + | select * from v2 where c1 > 0 union select * from v2 where c2 > 0 + | ) + | ) + |) + |select count(*) from ( + | select * from v where c1 > 0 union select * from v where c2 > 0 + |) + """.stripMargin) + checkAnswer(df, Row(2) :: Nil) + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case r: ReusedExchangeExec => r + }.length == 2, + "Non-deterministic CTEs are reused shuffles.") + } + } + + test("SPARK-36447: nested deterministic CTEs are inlined") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val df = sql( + s"""with + |v1 as ( + | select c1, c2, c1 + c2 c3 from t + |), + |v2 as ( + | select * from v1 where c3 in (select c3 from v1) + |) + |select count(*) from ( + | select * from v2 where c1 > 0 union select * from v2 where c2 > 0 + |) + """.stripMargin) + checkAnswer(df, Row(2) :: Nil) + assert( + df.queryExecution.analyzed.collect { + case WithCTE(_, cteDefs) => cteDefs + }.head.length == 2, + "With-CTE should contain 2 CTE defs after analysis.") + assert( + df.queryExecution.optimizedPlan.collect { + case WithCTE(_, cteDefs) => cteDefs + }.isEmpty, + "Deterministic CTEs should all be inlined after optimization.") + } + } + + test("SPARK-36447: invalid nested CTEs") { + withView("t") { + Seq((0, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + val ex = intercept[AnalysisException](sql( + s"""with + |v2 as ( + | select * from v1 where c3 in (select c3 from v1) + |), + |v1 as ( + | select c1, c2, rand() c3 from t + |) + |select count(*) from ( + | select * from v2 where c1 > 0 union select * from v2 where c2 > 0 + |) + """.stripMargin)) + assert(ex.message.contains("Table or view not found: v1")) + } + } +} + +class CTEInlineSuiteAEOff extends CTEInlineSuiteBase with DisableAdaptiveExecutionSuite + +class CTEInlineSuiteAEOn extends CTEInlineSuiteBase with EnableAdaptiveExecutionSuite